You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by el...@apache.org on 2016/09/14 21:48:01 UTC

[01/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Repository: phoenix
Updated Branches:
  refs/heads/4.8-HBase-0.98 c8f2f48df -> 714c7d129
  refs/heads/4.8-HBase-1.0 56ea4ae0b -> ccd04ba68
  refs/heads/4.8-HBase-1.1 b14897bc1 -> 8e33134fe
  refs/heads/4.8-HBase-1.2 f070950c1 -> 0b0db6103
  refs/heads/4.x-HBase-0.98 9fbee8c79 -> 98ef14822
  refs/heads/4.x-HBase-1.1 f0d8ff882 -> 09ce1f512
  refs/heads/master 7af708873 -> 2c2b552cd


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2c2b552c/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 155d1ba..ace228b 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/2c2b552c/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[12/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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/0b0db610
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/0b0db610
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/0b0db610

Branch: refs/heads/4.8-HBase-1.2
Commit: 0b0db61038597661f6c62bd56674755cfea9aaf2
Parents: f070950
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 17:08:48 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1172 ++++++++++--------
 .../query/ConnectionQueryServicesImpl.java      | 1005 +++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1432 insertions(+), 1318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b0db610/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index b7537a6..41d2f60 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -45,10 +46,12 @@ import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
@@ -59,6 +62,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,162 +83,162 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseHBaseManagedTimeIT {
-	
-	private final boolean localIndex;
+    
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-	private final String tableName;
+    private final String tableDDLOptions;
+    private final String tableName;
     private final String indexName;
     private final String fullTableName;
     private final String fullIndexName;
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+    
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable) 
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
         this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	}
-	
-	@BeforeClass
+    }
+    
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
@@ -253,13 +257,13 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -283,39 +287,39 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -330,85 +334,91 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -419,19 +429,22 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -441,72 +454,77 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
+
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -515,7 +533,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -526,7 +544,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -535,303 +553,315 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	String testTable = fullTableName+"_MULTIPLE_UPDATES";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
-	        		  + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                      + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
+                      + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+    
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -851,7 +881,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -870,7 +900,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -899,9 +929,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -931,38 +961,45 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -970,7 +1007,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -985,7 +1022,54 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
-}
\ No newline at end of file
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
+    }
+
+}


[13/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8e33134f/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 155d1ba..ace228b 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/8e33134f/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[09/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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/98ef1482
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/98ef1482
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/98ef1482

Branch: refs/heads/4.x-HBase-0.98
Commit: 98ef148220e8a4473b6b78161318712eb1ed55e9
Parents: 9fbee8c
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 16:37:37 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1136 +++++++++---------
 .../query/ConnectionQueryServicesImpl.java      | 1008 ++++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1400 insertions(+), 1317 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/98ef1482/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index 072e216..a5fefe2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -43,11 +44,14 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
@@ -59,6 +63,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,45 +84,45 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseOwnClusterIT {
-	
-	private final boolean localIndex;
+
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-	}
-	
-	@BeforeClass
+    private final String tableDDLOptions;
+
+
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable)
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+    }
+
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
@@ -125,58 +130,58 @@ public class IndexIT extends BaseOwnClusterIT {
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -185,57 +190,57 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         String tableName = "TBL_" + generateRandomString();
@@ -258,13 +263,13 @@ public class IndexIT extends BaseOwnClusterIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -288,39 +293,39 @@ public class IndexIT extends BaseOwnClusterIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -335,7 +340,7 @@ public class IndexIT extends BaseOwnClusterIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -346,85 +351,85 @@ public class IndexIT extends BaseOwnClusterIT {
 
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -435,22 +440,22 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -460,9 +465,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -471,69 +476,69 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -542,7 +547,7 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -553,7 +558,7 @@ public class IndexIT extends BaseOwnClusterIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -562,9 +567,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -573,205 +578,205 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
@@ -779,107 +784,107 @@ public class IndexIT extends BaseOwnClusterIT {
 
         String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
-	              + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
+                    + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -899,7 +904,7 @@ public class IndexIT extends BaseOwnClusterIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -918,7 +923,7 @@ public class IndexIT extends BaseOwnClusterIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -947,9 +952,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -979,45 +984,45 @@ public class IndexIT extends BaseOwnClusterIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -1025,7 +1030,7 @@ public class IndexIT extends BaseOwnClusterIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -1040,7 +1045,54 @@ public class IndexIT extends BaseOwnClusterIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
+    }
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
     }
-    
-}
\ No newline at end of file
+
+}


[08/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/98ef1482/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 352aa50..79ac809 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
@@ -40,6 +40,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -257,6 +259,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -268,7 +271,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported() {
                 @Override
@@ -284,11 +287,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
-    
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -346,7 +349,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -354,13 +357,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -368,7 +371,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -381,7 +384,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -395,7 +398,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -405,11 +408,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -522,10 +525,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -537,7 +540,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -558,7 +561,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -615,22 +618,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -642,7 +645,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -684,7 +687,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -695,7 +698,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -718,7 +721,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -726,14 +729,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -775,9 +778,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -794,7 +819,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -834,7 +859,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -856,7 +881,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -865,7 +890,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -983,7 +1008,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1007,7 +1032,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1049,12 +1074,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1103,18 +1128,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
-    	try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+        try (HBaseAdmin admin = getAdmin()) {
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1187,18 +1212,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1222,12 +1247,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1267,10 +1293,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1328,11 +1350,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1360,10 +1382,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1402,30 +1424,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1436,26 +1458,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1469,26 +1491,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1512,7 +1534,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1542,28 +1564,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1608,7 +1630,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1620,7 +1642,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1634,8 +1656,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1659,13 +1681,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1686,7 +1712,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1696,13 +1722,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1711,9 +1737,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1721,37 +1747,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1857,7 +1883,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1886,7 +1912,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1915,7 +1941,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1939,7 +1965,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1951,7 +1977,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1973,13 +1999,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2010,8 +2036,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2029,7 +2055,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2043,7 +2069,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2067,7 +2093,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2084,7 +2110,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2127,7 +2153,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2142,7 +2168,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2154,12 +2180,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2172,7 +2198,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2182,25 +2208,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2217,7 +2243,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2250,15 +2276,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2291,31 +2317,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
-                            }   
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
+                            }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
@@ -2355,10 +2381,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2381,7 +2407,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2417,17 +2443,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
+                                        metaConnection = dropStatsTable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
                                                 PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                                 PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
@@ -2464,7 +2490,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2481,57 +2507,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                               metaConnection,
+                                               SYSTEM_STATS_NAME,
+                                               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                               PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                       + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2654,7 +2680,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + (snapshotRestored ? " after successfully restoring using snapshot"
                                                     + snapshotName
                                                     : " after restoring using snapshot "
-                                                            + snapshotName + " failed. "));
+                                                    + snapshotName + " failed. "));
                                 } finally {
                                     try {
                                         admin.close();
@@ -2675,7 +2701,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 private void ensureSystemTablesUpgraded(ReadOnlyProps props)
                         throws SQLException, IOException, IllegalArgumentException, InterruptedException {
                     if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }
@@ -2717,7 +2743,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 /**
                  * Acquire distributed mutex of sorts to make sure only one JVM is able to run the upgrade code by
                  * making use of HBase's checkAndPut api.
@@ -2728,13 +2754,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                  * wins the race will end up setting the version cell to the {@value MetaDataProtocol#MIN_SYSTEM_TABLE_TIMESTAMP}
                  * for the release.
                  * </p>
-                 * 
+                 *
                  * @return true if client won the race, false otherwise
                  * @throws IOException
                  * @throws SQLException
                  */
                 private boolean acquireUpgradeMutex(long currentServerSideTableTimestamp, byte[] sysCatalogTableName) throws IOException,
-                        SQLException {
+                SQLException {
                     Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
                     try (HTableInterface sysCatalogTable = getTable(sysCatalogTableName)) {
                         byte[] row = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
@@ -2760,7 +2786,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw Throwables.propagate(e);
         }
     }
-    
+
     private static class UpgradeInProgressException extends SQLException {
         public UpgradeInProgressException(String upgradeFrom, String upgradeTo) {
             super("Cluster is being concurrently upgraded from " + upgradeFrom + " to " + upgradeTo
@@ -2776,7 +2802,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2789,12 +2815,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }   
@@ -2813,18 +2839,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2864,8 +2890,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2888,46 +2914,46 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
-    
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
+
     private static int getSaltBuckets(TableAlreadyExistsException e) {
         PTable table = e.getTable();
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2957,22 +2983,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                                instance.clearCache(controller,

<TRUNCATED>

[15/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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

Branch: refs/heads/4.8-HBase-1.1
Commit: 8e33134fe0af53cd9820f9a32139afe07f5f5db1
Parents: b14897b
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 17:09:00 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1172 ++++++++++--------
 .../query/ConnectionQueryServicesImpl.java      | 1005 +++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1432 insertions(+), 1318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/8e33134f/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index b7537a6..41d2f60 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -45,10 +46,12 @@ import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
@@ -59,6 +62,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,162 +83,162 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseHBaseManagedTimeIT {
-	
-	private final boolean localIndex;
+    
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-	private final String tableName;
+    private final String tableDDLOptions;
+    private final String tableName;
     private final String indexName;
     private final String fullTableName;
     private final String fullIndexName;
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+    
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable) 
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
         this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	}
-	
-	@BeforeClass
+    }
+    
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
@@ -253,13 +257,13 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -283,39 +287,39 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -330,85 +334,91 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -419,19 +429,22 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -441,72 +454,77 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
+
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -515,7 +533,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -526,7 +544,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -535,303 +553,315 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	String testTable = fullTableName+"_MULTIPLE_UPDATES";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
-	        		  + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                      + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
+                      + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+    
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -851,7 +881,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -870,7 +900,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -899,9 +929,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -931,38 +961,45 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -970,7 +1007,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -985,7 +1022,54 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
-}
\ No newline at end of file
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
+    }
+
+}


[02/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/2c2b552c/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 607ad5e..767a600 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
@@ -39,6 +39,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -256,7 +258,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
-    
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -268,7 +270,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported() {
                 @Override
@@ -284,11 +286,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
-    
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -346,7 +348,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -354,13 +356,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -368,7 +370,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -381,7 +383,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -395,7 +397,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -405,11 +407,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -522,10 +524,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -537,7 +539,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -558,7 +560,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -615,22 +617,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -642,7 +644,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -683,7 +685,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -694,7 +696,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -717,7 +719,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -725,14 +727,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -774,9 +776,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -793,7 +817,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -833,7 +857,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -855,7 +879,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -864,7 +888,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -982,7 +1006,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1006,7 +1030,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1048,12 +1072,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1103,17 +1127,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
         try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1186,18 +1210,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1221,12 +1245,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1266,10 +1291,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1327,11 +1348,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1359,10 +1380,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1401,30 +1422,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1435,26 +1456,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1468,26 +1489,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1511,7 +1532,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1541,28 +1562,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1612,7 +1633,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1624,7 +1645,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1638,8 +1659,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1663,13 +1684,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1690,7 +1715,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1700,13 +1725,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1715,9 +1740,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1725,37 +1750,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1861,7 +1886,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1890,7 +1915,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1919,7 +1944,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1943,7 +1968,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1955,7 +1980,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1977,13 +2002,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2014,8 +2039,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2033,7 +2058,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2047,7 +2072,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2071,7 +2096,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2088,7 +2113,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2131,7 +2156,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2146,7 +2171,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2158,12 +2183,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2176,7 +2201,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2186,25 +2211,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2221,7 +2246,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2254,15 +2279,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2292,34 +2317,34 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         try {
                             openConnection();
                             String noUpgradeProp = props.getProperty(PhoenixRuntime.NO_UPGRADE_ATTRIB);
-                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp)); 
+                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
-                            }   
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
+                            }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
@@ -2359,10 +2384,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2385,7 +2410,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2421,17 +2446,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
+                                        metaConnection = dropStatsTable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
                                                 PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                                 PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
@@ -2468,7 +2493,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2485,57 +2510,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                               metaConnection,
+                                               SYSTEM_STATS_NAME,
+                                               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                               PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                       + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2658,7 +2683,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + (snapshotRestored ? " after successfully restoring using snapshot"
                                                     + snapshotName
                                                     : " after restoring using snapshot "
-                                                            + snapshotName + " failed. "));
+                                                    + snapshotName + " failed. "));
                                 } finally {
                                     try {
                                         admin.close();
@@ -2679,7 +2704,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 private void ensureSystemTablesUpgraded(ReadOnlyProps props)
                         throws SQLException, IOException, IllegalArgumentException, InterruptedException {
                     if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }
@@ -2721,7 +2746,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 /**
                  * Acquire distributed mutex of sorts to make sure only one JVM is able to run the upgrade code by
                  * making use of HBase's checkAndPut api.
@@ -2732,13 +2757,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                  * wins the race will end up setting the version cell to the {@value MetaDataProtocol#MIN_SYSTEM_TABLE_TIMESTAMP}
                  * for the release.
                  * </p>
-                 * 
+                 *
                  * @return true if client won the race, false otherwise
                  * @throws IOException
                  * @throws SQLException
                  */
                 private boolean acquireUpgradeMutex(long currentServerSideTableTimestamp, byte[] sysCatalogTableName) throws IOException,
-                        SQLException {
+                SQLException {
                     Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
                     try (HTableInterface sysCatalogTable = getTable(sysCatalogTableName)) {
                         byte[] row = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
@@ -2764,7 +2789,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw Throwables.propagate(e);
         }
     }
-    
+
     private static class UpgradeInProgressException extends SQLException {
         public UpgradeInProgressException(String upgradeFrom, String upgradeTo) {
             super("Cluster is being concurrently upgraded from " + upgradeFrom + " to " + upgradeTo
@@ -2780,7 +2805,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2803,18 +2828,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2854,8 +2879,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2878,50 +2903,50 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
 
     private void scheduleRenewLeaseTasks() {
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }
@@ -2931,7 +2956,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2961,22 +2986,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                bui

<TRUNCATED>

[16/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/ccd04ba6/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 a83d4d3..b3a5a36 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ccd04ba6/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[11/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b0db610/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 607ad5e..767a600 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
@@ -39,6 +39,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -256,7 +258,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
-    
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -268,7 +270,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported() {
                 @Override
@@ -284,11 +286,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
-    
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -346,7 +348,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -354,13 +356,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -368,7 +370,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -381,7 +383,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -395,7 +397,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -405,11 +407,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -522,10 +524,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -537,7 +539,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -558,7 +560,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -615,22 +617,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -642,7 +644,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -683,7 +685,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -694,7 +696,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -717,7 +719,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -725,14 +727,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -774,9 +776,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -793,7 +817,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -833,7 +857,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -855,7 +879,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -864,7 +888,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -982,7 +1006,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1006,7 +1030,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1048,12 +1072,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1103,17 +1127,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
         try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1186,18 +1210,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1221,12 +1245,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1266,10 +1291,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1327,11 +1348,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1359,10 +1380,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1401,30 +1422,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1435,26 +1456,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1468,26 +1489,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1511,7 +1532,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1541,28 +1562,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1612,7 +1633,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1624,7 +1645,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1638,8 +1659,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1663,13 +1684,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1690,7 +1715,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1700,13 +1725,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1715,9 +1740,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1725,37 +1750,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1861,7 +1886,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1890,7 +1915,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1919,7 +1944,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1943,7 +1968,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1955,7 +1980,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1977,13 +2002,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2014,8 +2039,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2033,7 +2058,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2047,7 +2072,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2071,7 +2096,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2088,7 +2113,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2131,7 +2156,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2146,7 +2171,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2158,12 +2183,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2176,7 +2201,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2186,25 +2211,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2221,7 +2246,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2254,15 +2279,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2292,34 +2317,34 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         try {
                             openConnection();
                             String noUpgradeProp = props.getProperty(PhoenixRuntime.NO_UPGRADE_ATTRIB);
-                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp)); 
+                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
-                            }   
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
+                            }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
@@ -2359,10 +2384,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2385,7 +2410,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2421,17 +2446,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
+                                        metaConnection = dropStatsTable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
                                                 PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                                 PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
@@ -2468,7 +2493,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2485,57 +2510,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                               metaConnection,
+                                               SYSTEM_STATS_NAME,
+                                               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                               PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                       + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2658,7 +2683,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + (snapshotRestored ? " after successfully restoring using snapshot"
                                                     + snapshotName
                                                     : " after restoring using snapshot "
-                                                            + snapshotName + " failed. "));
+                                                    + snapshotName + " failed. "));
                                 } finally {
                                     try {
                                         admin.close();
@@ -2679,7 +2704,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 private void ensureSystemTablesUpgraded(ReadOnlyProps props)
                         throws SQLException, IOException, IllegalArgumentException, InterruptedException {
                     if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }
@@ -2721,7 +2746,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 /**
                  * Acquire distributed mutex of sorts to make sure only one JVM is able to run the upgrade code by
                  * making use of HBase's checkAndPut api.
@@ -2732,13 +2757,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                  * wins the race will end up setting the version cell to the {@value MetaDataProtocol#MIN_SYSTEM_TABLE_TIMESTAMP}
                  * for the release.
                  * </p>
-                 * 
+                 *
                  * @return true if client won the race, false otherwise
                  * @throws IOException
                  * @throws SQLException
                  */
                 private boolean acquireUpgradeMutex(long currentServerSideTableTimestamp, byte[] sysCatalogTableName) throws IOException,
-                        SQLException {
+                SQLException {
                     Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
                     try (HTableInterface sysCatalogTable = getTable(sysCatalogTableName)) {
                         byte[] row = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
@@ -2764,7 +2789,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw Throwables.propagate(e);
         }
     }
-    
+
     private static class UpgradeInProgressException extends SQLException {
         public UpgradeInProgressException(String upgradeFrom, String upgradeTo) {
             super("Cluster is being concurrently upgraded from " + upgradeFrom + " to " + upgradeTo
@@ -2780,7 +2805,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2803,18 +2828,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2854,8 +2879,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2878,50 +2903,50 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
 
     private void scheduleRenewLeaseTasks() {
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }
@@ -2931,7 +2956,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2961,22 +2986,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                bui

<TRUNCATED>

[10/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b0db610/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 155d1ba..ace228b 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b0db610/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[17/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/ccd04ba6/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 9aa4b5d..dbbc37b 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
@@ -34,6 +34,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -79,6 +80,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -250,6 +252,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -263,7 +266,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported(){
                 @Override
@@ -278,12 +281,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     int hbaseVersion = services.getLowestClusterHBaseVersion();
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
-             });
-    
+            });
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -341,7 +344,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -349,13 +352,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -363,7 +366,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -376,7 +379,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -390,7 +393,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -400,11 +403,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -517,10 +520,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -532,7 +535,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -553,7 +556,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -610,22 +613,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -637,7 +640,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -678,7 +681,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -689,7 +692,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -712,7 +715,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -720,14 +723,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -769,9 +772,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -788,7 +813,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -828,7 +853,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -850,7 +875,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -859,7 +884,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -977,7 +1002,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1001,7 +1026,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1043,12 +1068,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1097,18 +1122,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
-    	try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+        try (HBaseAdmin admin = getAdmin()) {
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1181,18 +1206,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1216,12 +1241,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1261,10 +1287,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1322,11 +1344,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1354,10 +1376,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1396,30 +1418,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1430,26 +1452,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1463,26 +1485,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1506,7 +1528,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1536,28 +1558,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1602,7 +1624,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1614,7 +1636,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1628,8 +1650,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1653,13 +1675,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1680,7 +1706,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1690,13 +1716,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1705,9 +1731,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1715,37 +1741,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1851,7 +1877,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1880,7 +1906,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1909,7 +1935,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1933,7 +1959,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1945,7 +1971,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1967,13 +1993,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2004,8 +2030,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2023,7 +2049,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2037,7 +2063,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2061,7 +2087,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2078,7 +2104,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2121,7 +2147,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2136,7 +2162,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2148,12 +2174,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2166,7 +2192,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2176,25 +2202,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2211,7 +2237,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2244,15 +2270,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2282,32 +2308,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
                             }
-
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
 
@@ -2344,10 +2369,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                             + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2370,7 +2395,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2406,18 +2431,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
-                                            PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
-                                            PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
+                                        metaConnection = dropStatsTable(metaConnection,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
+                                                PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
+                                                PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
                                     }
@@ -2452,7 +2477,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2469,57 +2494,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            PhoenixDatabaseMetaData.SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                                metaConnection,
+                                                PhoenixDatabaseMetaData.SYSTEM_STATS_NAME,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                                PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                        + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2611,7 +2636,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2624,12 +2649,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }   
@@ -2648,18 +2673,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2699,8 +2724,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2723,46 +2748,46 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
-    
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
+
     private static int getSaltBuckets(TableAlreadyExistsException e) {
         PTable table = e.getTable();
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2792,22 +2817,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                                instance.clearCache(controller, builder.build(), rpcCallback);
+                                if(controller.getFailedOn() != null) {
+                                    throw controller.getFailedOn();
+                                }
+                                return rpcCallback.get().getUnfreedBytes();
                             }
-                            return rpcCallback.get().getUnfreedBytes();
-                        }
-                      });
+                        });
                 long unfreedBytes = 0;
                 for (Map.Entry<byte[],Long> result : results.entrySet()) {
                     if (result.getValue() != null) {
@@ -2851,7 +2876,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // restore the interrupt status
             Thread.currentThread().interrupt();
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION).setRootCause(e).build()
-                    .buildException();
+            .buildException();
         } finally {
             Closeables.closeQuietly(admin);
         }
@@ -3088,21 +3113,21 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
                         new Batch.Call<MetaDataService, ClearTableFromCacheResponse>() {
-                            @Override
-                            public ClearTableFromCacheResponse call(MetaDataService instance) throws IOException {
-                                ServerRpcController controller = new ServerRpcController();
-                                BlockingRpcCallback<ClearTableFromCacheResponse> rpcCallback = new BlockingRpcCallback<ClearTableFromCacheResponse>();
-                                ClearTableFromCacheRequest.Builder builder = ClearTableFromCacheRequest.newBuilder();
-                                builder.setTenantId(ByteStringer.wrap(tenantId));
-                                builder.setTableName(ByteStringer.wrap(tableName));
-                                builder.setSchemaName(ByteStringer.wrap(schemaName));
- 

<TRUNCATED>

[06/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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/09ce1f51
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/09ce1f51
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/09ce1f51

Branch: refs/heads/4.x-HBase-1.1
Commit: 09ce1f51280a7c6af2b8995877be294b62feeb60
Parents: f0d8ff8
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 14:45:56 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1136 +++++++++---------
 .../query/ConnectionQueryServicesImpl.java      | 1005 ++++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1398 insertions(+), 1316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/09ce1f51/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index 072e216..a5fefe2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -43,11 +44,14 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
@@ -59,6 +63,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,45 +84,45 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseOwnClusterIT {
-	
-	private final boolean localIndex;
+
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-	}
-	
-	@BeforeClass
+    private final String tableDDLOptions;
+
+
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable)
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+    }
+
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
@@ -125,58 +130,58 @@ public class IndexIT extends BaseOwnClusterIT {
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -185,57 +190,57 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         String tableName = "TBL_" + generateRandomString();
@@ -258,13 +263,13 @@ public class IndexIT extends BaseOwnClusterIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -288,39 +293,39 @@ public class IndexIT extends BaseOwnClusterIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -335,7 +340,7 @@ public class IndexIT extends BaseOwnClusterIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -346,85 +351,85 @@ public class IndexIT extends BaseOwnClusterIT {
 
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -435,22 +440,22 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -460,9 +465,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -471,69 +476,69 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -542,7 +547,7 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -553,7 +558,7 @@ public class IndexIT extends BaseOwnClusterIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -562,9 +567,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -573,205 +578,205 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
@@ -779,107 +784,107 @@ public class IndexIT extends BaseOwnClusterIT {
 
         String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
-	              + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
+                    + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -899,7 +904,7 @@ public class IndexIT extends BaseOwnClusterIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -918,7 +923,7 @@ public class IndexIT extends BaseOwnClusterIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -947,9 +952,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -979,45 +984,45 @@ public class IndexIT extends BaseOwnClusterIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -1025,7 +1030,7 @@ public class IndexIT extends BaseOwnClusterIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -1040,7 +1045,54 @@ public class IndexIT extends BaseOwnClusterIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
+    }
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
     }
-    
-}
\ No newline at end of file
+
+}


[19/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/714c7d12/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 155d1ba..ace228b 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/714c7d12/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[03/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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/2c2b552c
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/2c2b552c
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/2c2b552c

Branch: refs/heads/master
Commit: 2c2b552cd7c611c63677b0be075065504d579469
Parents: 7af7088
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 14:44:04 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1136 +++++++++---------
 .../query/ConnectionQueryServicesImpl.java      | 1005 ++++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1398 insertions(+), 1316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/2c2b552c/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index 072e216..a5fefe2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -43,11 +44,14 @@ import java.util.Properties;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeTableReuseIT;
@@ -59,6 +63,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,45 +84,45 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseOwnClusterIT {
-	
-	private final boolean localIndex;
+
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-	}
-	
-	@BeforeClass
+    private final String tableDDLOptions;
+
+
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable)
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+    }
+
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeTableReuseIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
@@ -125,58 +130,58 @@ public class IndexIT extends BaseOwnClusterIT {
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
 
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -185,57 +190,57 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         String tableName = "TBL_" + generateRandomString();
@@ -258,13 +263,13 @@ public class IndexIT extends BaseOwnClusterIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -288,39 +293,39 @@ public class IndexIT extends BaseOwnClusterIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -335,7 +340,7 @@ public class IndexIT extends BaseOwnClusterIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -346,85 +351,85 @@ public class IndexIT extends BaseOwnClusterIT {
 
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -435,22 +440,22 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -460,9 +465,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -471,69 +476,69 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -542,7 +547,7 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -553,7 +558,7 @@ public class IndexIT extends BaseOwnClusterIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -562,9 +567,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -573,205 +578,205 @@ public class IndexIT extends BaseOwnClusterIT {
         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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
@@ -779,107 +784,107 @@ public class IndexIT extends BaseOwnClusterIT {
 
         String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
-	              + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
+                    + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -899,7 +904,7 @@ public class IndexIT extends BaseOwnClusterIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -918,7 +923,7 @@ public class IndexIT extends BaseOwnClusterIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -947,9 +952,9 @@ public class IndexIT extends BaseOwnClusterIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -979,45 +984,45 @@ public class IndexIT extends BaseOwnClusterIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String tableName = "TBL_" + generateRandomString();
         String indexName = "IND_" + generateRandomString();
         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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             TestUtil.createMultiCFTestTable(conn, fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -1025,7 +1030,7 @@ public class IndexIT extends BaseOwnClusterIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -1040,7 +1045,54 @@ public class IndexIT extends BaseOwnClusterIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
+    }
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
     }
-    
-}
\ No newline at end of file
+
+}


[21/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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/714c7d12
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/714c7d12
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/714c7d12

Branch: refs/heads/4.8-HBase-0.98
Commit: 714c7d129e5c8dc3bebcb09d21597462dbbbfc1b
Parents: c8f2f48
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 17:47:24 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1172 ++++++++++--------
 .../query/ConnectionQueryServicesImpl.java      |  994 +++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1427 insertions(+), 1312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/714c7d12/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index b7537a6..41d2f60 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -45,10 +46,12 @@ import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
@@ -59,6 +62,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,162 +83,162 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseHBaseManagedTimeIT {
-	
-	private final boolean localIndex;
+    
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-	private final String tableName;
+    private final String tableDDLOptions;
+    private final String tableName;
     private final String indexName;
     private final String fullTableName;
     private final String fullIndexName;
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+    
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable) 
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
         this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	}
-	
-	@BeforeClass
+    }
+    
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
@@ -253,13 +257,13 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -283,39 +287,39 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -330,85 +334,91 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -419,19 +429,22 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -441,72 +454,77 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
+
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -515,7 +533,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -526,7 +544,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -535,303 +553,315 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	String testTable = fullTableName+"_MULTIPLE_UPDATES";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
-	        		  + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                      + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
+                      + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+    
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -851,7 +881,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -870,7 +900,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -899,9 +929,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -931,38 +961,45 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -970,7 +1007,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -985,7 +1022,54 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
-}
\ No newline at end of file
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
+    }
+
+}


[07/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/98ef1482/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 155d1ba..ace228b 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/98ef1482/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[04/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/09ce1f51/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 155d1ba..ace228b 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
@@ -168,6 +168,11 @@ public interface QueryConstants {
 
     public static final byte[] TRUE = new byte[] {1};
     
+    /**
+     * The priority property for an hbase table. This is already in HTD, but older versions of
+     * HBase do not have this, so we re-defined it here. Once Phoenix is HBase-1.3+, we can remote.
+     */
+    public static final String PRIORITY = "PRIORITY";
 
     /**
      * Separator used between variable length keys for a composite key.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/09ce1f51/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 7da7010..28ed11d 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
@@ -228,168 +228,168 @@ public class MetaDataClient {
     private static final ParseNodeFactory FACTORY = new ParseNodeFactory();
     private static final String SET_ASYNC_CREATED_DATE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    ASYNC_CREATED_DATE + " " + PDate.INSTANCE.getSqlTypeName() +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String CREATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_TYPE + "," +
-            TABLE_SEQ_NUM + "," +
-            COLUMN_COUNT + "," +
-            SALT_BUCKETS + "," +
-            PK_NAME + "," +
-            DATA_TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            IMMUTABLE_ROWS + "," +
-            DEFAULT_COLUMN_FAMILY_NAME + "," +
-            VIEW_STATEMENT + "," +
-            DISABLE_WAL + "," +
-            MULTI_TENANT + "," +
-            VIEW_TYPE + "," +
-            VIEW_INDEX_ID + "," +
-            INDEX_TYPE + "," +
-            STORE_NULLS + "," +
-            BASE_COLUMN_COUNT + "," +
-            TRANSACTIONAL + "," +
-            UPDATE_CACHE_FREQUENCY + "," +
-            IS_NAMESPACE_MAPPED + "," +
-            AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT + "," +
+                    SALT_BUCKETS + "," +
+                    PK_NAME + "," +
+                    DATA_TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    IMMUTABLE_ROWS + "," +
+                    DEFAULT_COLUMN_FAMILY_NAME + "," +
+                    VIEW_STATEMENT + "," +
+                    DISABLE_WAL + "," +
+                    MULTI_TENANT + "," +
+                    VIEW_TYPE + "," +
+                    VIEW_INDEX_ID + "," +
+                    INDEX_TYPE + "," +
+                    STORE_NULLS + "," +
+                    BASE_COLUMN_COUNT + "," +
+                    TRANSACTIONAL + "," +
+                    UPDATE_CACHE_FREQUENCY + "," +
+                    IS_NAMESPACE_MAPPED + "," +
+                    AUTO_PARTITION_SEQ +  "," +
+                    APPEND_ONLY_SCHEMA +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
     private static final String CREATE_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
-            TABLE_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
+                    TABLE_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?)";
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_FAMILY + "," +
-            LINK_TYPE + "," +
-            PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    LINK_TYPE + "," +
+                    PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            TABLE_SEQ_NUM  +
-            ") VALUES (?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_SEQ_NUM  +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String MUTATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        TABLE_TYPE + "," +
-        TABLE_SEQ_NUM + "," +
-        COLUMN_COUNT +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    TABLE_TYPE + "," +
+                    TABLE_SEQ_NUM + "," +
+                    COLUMN_COUNT +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        INDEX_STATE +
-        ") VALUES (?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE +
+                    ") VALUES (?, ?, ?, ?)";
     private static final String UPDATE_INDEX_STATE_TO_ACTIVE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            INDEX_STATE + "," +
-            INDEX_DISABLE_TIMESTAMP +
-            ") VALUES (?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    INDEX_STATE + "," +
+                    INDEX_DISABLE_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?)";
     //TODO: merge INSERT_COLUMN_CREATE_TABLE and INSERT_COLUMN_ALTER_TABLE column when
     // the new major release is out.
     private static final String INSERT_COLUMN_CREATE_TABLE =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        DATA_TYPE + "," +
-        NULLABLE + "," +
-        COLUMN_SIZE + "," +
-        DECIMAL_DIGITS + "," +
-        ORDINAL_POSITION + "," +
-        SORT_ORDER + "," +
-        DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-        ARRAY_SIZE + "," +
-        VIEW_CONSTANT + "," +
-        IS_VIEW_REFERENCED + "," +
-        PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-        KEY_SEQ + "," +
-        COLUMN_DEF + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF + "," +
+                    IS_ROW_TIMESTAMP +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-            TENANT_ID + "," +
-            TABLE_SCHEM + "," +
-            TABLE_NAME + "," +
-            COLUMN_NAME + "," +
-            COLUMN_FAMILY + "," +
-            DATA_TYPE + "," +
-            NULLABLE + "," +
-            COLUMN_SIZE + "," +
-            DECIMAL_DIGITS + "," +
-            ORDINAL_POSITION + "," +
-            SORT_ORDER + "," +
-            DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
-            ARRAY_SIZE + "," +
-            VIEW_CONSTANT + "," +
-            IS_VIEW_REFERENCED + "," +
-            PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
-            KEY_SEQ + "," +
-            COLUMN_DEF +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
+                    COLUMN_DEF +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
-        "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
-        TENANT_ID + "," +
-        TABLE_SCHEM + "," +
-        TABLE_NAME + "," +
-        COLUMN_NAME + "," +
-        COLUMN_FAMILY + "," +
-        ORDINAL_POSITION +
-        ") VALUES (?, ?, ?, ?, ?, ?)";
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    ORDINAL_POSITION +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String CREATE_FUNCTION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            NUM_ARGS + "," +
-            CLASS_NAME + "," +
-            JAR_PATH + "," +
-            RETURN_TYPE +
-            ") VALUES (?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    NUM_ARGS + "," +
+                    CLASS_NAME + "," +
+                    JAR_PATH + "," +
+                    RETURN_TYPE +
+                    ") VALUES (?, ?, ?, ?, ?, ?)";
     private static final String INSERT_FUNCTION_ARGUMENT =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_FUNCTION_TABLE + "\" ( " +
-            TENANT_ID +","+
-            FUNCTION_NAME + "," +
-            TYPE + "," +
-            ARG_POSITION +","+
-            IS_ARRAY + "," +
-            IS_CONSTANT  + "," +
-            DEFAULT_VALUE + "," +
-            MIN_VALUE + "," +
-            MAX_VALUE +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    TENANT_ID +","+
+                    FUNCTION_NAME + "," +
+                    TYPE + "," +
+                    ARG_POSITION +","+
+                    IS_ARRAY + "," +
+                    IS_CONSTANT  + "," +
+                    DEFAULT_VALUE + "," +
+                    MIN_VALUE + "," +
+                    MAX_VALUE +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     public static final String EMPTY_TABLE = " ";
 
@@ -408,7 +408,7 @@ public class MetaDataClient {
         MetaDataMutationResult result = updateCache(schemaName, tableName, true);
         return result.getMutationTime();
     }
-    
+
     /**
      * Update the cache with the latest as of the connection scn.
      * @param schemaName
@@ -427,7 +427,7 @@ public class MetaDataClient {
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, false);
     }
-    
+
     public MetaDataMutationResult updateCache(PName tenantId, String schemaName, String tableName, boolean alwaysHitServer) throws SQLException {
         return updateCache(tenantId, schemaName, tableName, alwaysHitServer, null);
     }
@@ -455,13 +455,13 @@ public class MetaDataClient {
         long clientTimeStamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return clientTimeStamp;
     }
-    
+
     private long getCurrentScn() {
         Long scn = connection.getSCN();
         long currentScn = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
         return currentScn;
     }
-    
+
     private MetaDataMutationResult updateCache(PName origTenantId, String schemaName, String tableName,
             boolean alwaysHitServer, Long resolvedTimestamp) throws SQLException { // TODO: pass byte[] herez
         boolean systemTable = SYSTEM_CATALOG_SCHEMA.equals(schemaName);
@@ -479,18 +479,18 @@ public class MetaDataClient {
             tableResolvedTimestamp = tableRef.getResolvedTimeStamp();
         } catch (TableNotFoundException e) {
         }
-        
+
         boolean defaultTransactional = connection.getQueryServices().getProps().getBoolean(
-                                            QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                            QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
+                QueryServicesOptions.DEFAULT_TRANSACTIONAL);
         // start a txn if all table are transactional by default or if we found the table in the cache and it is transactional
-        // TODO if system tables become transactional remove the check 
+        // TODO if system tables become transactional remove the check
         boolean isTransactional = defaultTransactional || (table!=null && table.isTransactional());
         if (!systemTable && isTransactional && !connection.getMutationState().isTransactionStarted()) {
             connection.getMutationState().startTransaction();
         }
         resolvedTimestamp = resolvedTimestamp==null ? TransactionUtil.getResolvedTimestamp(connection, isTransactional, HConstants.LATEST_TIMESTAMP) : resolvedTimestamp;
-        // Do not make rpc to getTable if 
+        // Do not make rpc to getTable if
         // 1. table is a system table
         // 2. table was already resolved as of that timestamp
         if (table != null && !alwaysHitServer
@@ -507,7 +507,7 @@ public class MetaDataClient {
             final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
             ConnectionQueryServices queryServices = connection.getQueryServices();
             result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
-            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa) 
+            // if the table was assumed to be transactional, but is actually not transactional then re-resolve as of the right timestamp (and vice versa)
             if (table==null && result.getTable()!=null && result.getTable().isTransactional()!=isTransactional) {
                 result = queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, TransactionUtil.getResolvedTimestamp(connection, result.getTable().isTransactional(), HConstants.LATEST_TIMESTAMP));
             }
@@ -545,7 +545,7 @@ public class MetaDataClient {
                             connection.addTable(result.getTable(), resolvedTime);
                         }
                         else {
-                            // if we aren't adding the table, we still need to update the resolved time of the table 
+                            // if we aren't adding the table, we still need to update the resolved time of the table
                             connection.updateResolvedTimestamp(table, resolvedTime);
                         }
                         return result;
@@ -596,7 +596,7 @@ public class MetaDataClient {
                 String functionName = iterator.next();
                 function =
                         connection.getMetaDataCache().getFunction(
-                            new PTableKey(tenantId, functionName));
+                                new PTableKey(tenantId, functionName));
                 if (function != null && !alwaysHitServer
                         && function.getTimeStamp() == clientTimeStamp - 1) {
                     functions.add(function);
@@ -622,7 +622,7 @@ public class MetaDataClient {
         MetaDataMutationResult result;
 
         do {
-            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size()); 
+            List<Pair<byte[], Long>> functionsToFecth = new ArrayList<Pair<byte[], Long>>(functionNames.size());
             for(int i = 0; i< functionNames.size(); i++) {
                 functionsToFecth.add(new Pair<byte[], Long>(PVarchar.INSTANCE.toBytes(functionNames.get(i)), functionTimeStamps.get(i)));
             }
@@ -643,9 +643,9 @@ public class MetaDataClient {
                 if (code == MutationCode.FUNCTION_NOT_FOUND && tryCount + 1 == maxTryCount) {
                     for (Pair<byte[], Long> f : functionsToFecth) {
                         connection.removeFunction(tenantId, Bytes.toString(f.getFirst()),
-                            f.getSecond());
+                                f.getSecond());
                     }
-                    // TODO removeFunctions all together from cache when 
+                    // TODO removeFunctions all together from cache when
                     throw new FunctionNotFoundException(functionNames.toString() + " not found");
                 }
             }
@@ -721,7 +721,7 @@ public class MetaDataClient {
                 }
             }
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table. 
+            // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
                 if (col.getViewConstant() != null) {
                     try {
@@ -730,7 +730,7 @@ public class MetaDataClient {
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
                         index.getColumn(indexColumnName);
-                    } catch (ColumnNotFoundException e1) { 
+                    } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
@@ -755,10 +755,10 @@ public class MetaDataClient {
             if (containsAllReqdCols) {
                 // Tack on view statement to index to get proper filtering for view
                 String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR 
-                    + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table 
-                // also set update cache frequency to never since the renamed index is not present on the server 
+                PName modifiedIndexName = PNameFactory.newName(index.getSchemaName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR
+                        + index.getName().getString() + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + view.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // also set update cache frequency to never since the renamed index is not present on the server
                 indexesToAdd.add(PTableImpl.makePTable(index, modifiedIndexName, viewStatement, Long.MAX_VALUE, view.getTenantId()));
             }
         }
@@ -843,24 +843,24 @@ public class MetaDataClient {
             String columnName = columnDefName.getColumnName();
             if (isPK && sortOrder == SortOrder.DESC && def.getDataType() == PVarbinary.INSTANCE) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.DESC_VARBINARY_NOT_SUPPORTED)
-                    .setColumnName(columnName)
-                    .build().buildException();
+                .setColumnName(columnName)
+                .build().buildException();
             }
 
             PName familyName = null;
             if (def.isPK() && !pkConstraint.getColumnNames().isEmpty() ) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_ALREADY_EXISTS)
-                    .setColumnName(columnName).build().buildException();
+                .setColumnName(columnName).build().buildException();
             }
             boolean isNull = def.isNull();
             if (def.getColumnDefName().getFamilyName() != null) {
                 String family = def.getColumnDefName().getFamilyName();
                 if (isPK) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_WITH_FAMILY_NAME)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 } else if (!def.isNull()) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.KEY_VALUE_NOT_NULL)
-                        .setColumnName(columnName).setFamilyName(family).build().buildException();
+                    .setColumnName(columnName).setFamilyName(family).build().buildException();
                 }
                 familyName = PNameFactory.newName(family);
             } else if (!isPK) {
@@ -888,7 +888,7 @@ public class MetaDataClient {
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
-        
+
         boolean isAppendOnlySchema = false;
         Boolean appendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
         if (appendOnlySchemaProp != null) {
@@ -905,25 +905,25 @@ public class MetaDataClient {
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        // view isAppendOnlySchema property must match the parent table 
+        // view isAppendOnlySchema property must match the parent table
         if (parent!=null && isAppendOnlySchema!= parent.isAppendOnlySchema()) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_APPEND_ONLY_SCHEMA)
             .setSchemaName(tableName.getSchemaName()).setTableName(tableName.getTableName())
             .build().buildException();
         }
-        
+
         PTable table = null;
         // if the APPEND_ONLY_SCHEMA attribute is true first check if the table is present in the cache
         // if it is add columns that are not already present
         if (isAppendOnlySchema) {
-            // look up the table in the cache 
+            // look up the table in the cache
             MetaDataMutationResult result = updateCache(tableName.getSchemaName(), tableName.getTableName());
             if (result.getMutationCode()==MutationCode.TABLE_ALREADY_EXISTS) {
                 table = result.getTable();
                 if (!statement.ifNotExists()) {
                     throw new NewerTableAlreadyExistsException(tableName.getSchemaName(), tableName.getTableName(), table);
                 }
-                
+
                 List<ColumnDef> columnDefs = statement.getColumnDefs();
                 PrimaryKeyConstraint pkConstraint = statement.getPrimaryKeyConstraint();
                 // get the list of columns to add
@@ -932,13 +932,13 @@ public class MetaDataClient {
                         columnDef.setIsPK(true);
                     }
                 }
-                // if there are new columns to add 
+                // if there are new columns to add
                 return addColumn(table, columnDefs, statement.getProps(), statement.ifNotExists(),
-                    true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
+                        true, NamedTableNode.create(statement.getTableName()), statement.getTableType());
             }
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, null, null, null, tableProps, commonFamilyProps);
-            
+
         if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
             return new MutationState(0,connection);
         }
@@ -1057,7 +1057,7 @@ public class MetaDataClient {
              * since it may not represent a "real" table in the case of the view indexes of a base table.
              */
             PostDDLCompiler compiler = new PostDDLCompiler(connection);
-            //even if table is transactional, while calculating stats we scan the table non-transactionally to 
+            //even if table is transactional, while calculating stats we scan the table non-transactionally to
             //view all the data belonging to the table
             PTable nonTxnLogicalTable = new DelegateTable(logicalTable) {
                 @Override
@@ -1165,7 +1165,7 @@ public class MetaDataClient {
             } catch (IOException e) {
                 throw new SQLException(e);
             }
-            
+
             // execute index population upsert select
             long startTime = System.currentTimeMillis();
             MutationState state = connection.getQueryServices().updateData(mutationPlan);
@@ -1176,10 +1176,10 @@ public class MetaDataClient {
             // that were being written on the server while the index was created
             long sleepTime =
                     connection
-                            .getQueryServices()
-                            .getProps()
-                            .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
-                                QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
+                    .getQueryServices()
+                    .getProps()
+                    .getLong(QueryServices.INDEX_POPULATION_SLEEP_TIME,
+                        QueryServicesOptions.DEFAULT_INDEX_POPULATION_SLEEP_TIME);
             if (!dataTableRef.getTable().isTransactional() && sleepTime > 0) {
                 long delta = sleepTime - firstUpsertSelectTime;
                 if (delta > 0) {
@@ -1188,7 +1188,7 @@ public class MetaDataClient {
                     } catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                                .setRootCause(e).build().buildException();
+                        .setRootCause(e).build().buildException();
                     }
                 }
                 // set the min timestamp of second index upsert select some time before the index
@@ -1203,10 +1203,10 @@ public class MetaDataClient {
                         connection.getQueryServices().updateData(mutationPlan);
                 state.join(newMutationState);
             }
-            
+
             indexStatement = FACTORY.alterIndex(FACTORY.namedTable(null,
-            		TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
-            		dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
+            		    TableName.create(index.getSchemaName().getString(), index.getTableName().getString())),
+            		    dataTableRef.getTable().getTableName().getString(), false, PIndexState.ACTIVE);
             alterIndex(indexStatement);
 
             return state;
@@ -1246,7 +1246,7 @@ public class MetaDataClient {
     public MutationState createIndex(CreateIndexStatement statement, byte[][] splits) throws SQLException {
         IndexKeyConstraint ik = statement.getIndexConstraint();
         TableName indexTableName = statement.getIndexTableName();
-        
+
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
         Map<String,Object> commonFamilyProps = Maps.newHashMapWithExpectedSize(statement.getProps().size() + 1);
         populatePropertyMaps(statement.getProps(), tableProps, commonFamilyProps);
@@ -1305,7 +1305,7 @@ public class MetaDataClient {
                 }
                 List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
                 List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
-                
+
                 /*
                  * Allocate an index ID in two circumstances:
                  * 1) for a local index, as all local indexes will reside in the same HBase table
@@ -1318,7 +1318,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
                 }
-                
+
                 if (dataTable.isMultiTenant()) {
                     PColumn col = dataTable.getPKColumns().get(posOffset);
                     RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
@@ -1328,7 +1328,7 @@ public class MetaDataClient {
                     allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
                     columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
                 }
-                
+
                 PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
                 StatementContext context = new StatementContext(phoenixStatment, resolver);
                 IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
@@ -1339,7 +1339,7 @@ public class MetaDataClient {
                     parseNode = StatementNormalizer.normalize(parseNode, resolver);
                     // compile the parseNode to get an expression
                     expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);   
+                    Expression expression = parseNode.accept(expressionIndexCompiler);
                     if (expressionIndexCompiler.isAggregate()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
@@ -1350,25 +1350,25 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                     }
                     unusedPkColumns.remove(expression);
-                    
+
                     // Go through parse node to get string as otherwise we
                     // can lose information during compilation
                     StringBuilder buf = new StringBuilder();
                     parseNode.toSQL(resolver, buf);
                     // need to escape backslash as this expression will be re-parsed later
                     String expressionStr = StringUtil.escapeBackslash(buf.toString());
-                    
+
                     ColumnName colName = null;
                     ColumnRef colRef = expressionIndexCompiler.getColumnRef();
                     boolean isRowTimestamp = false;
-                    if (colRef!=null) { 
+                    if (colRef!=null) {
                         // if this is a regular column
                         PColumn column = colRef.getColumn();
                         String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
                         colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
                         isRowTimestamp = column.isRowTimestamp();
                     }
-                    else { 
+                    else {
                         // if this is an expression
                         // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
                         String name = expressionStr.replaceAll("\"", "'");
@@ -1396,7 +1396,7 @@ public class MetaDataClient {
                         }
                     }
                 }
-                
+
                 // Last all the included columns (minus any PK columns)
                 for (ColumnName colName : includedColumns) {
                     PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
@@ -1425,8 +1425,8 @@ public class MetaDataClient {
                     // if scn is set create at scn-1, so we can see the sequence or else use latest timestamp (so that latest server time is used)
                     long sequenceTimestamp = scn!=null ? scn-1 : HConstants.LATEST_TIMESTAMP;
                     createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                        true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
-                        sequenceTimestamp);
+                            true, Short.MIN_VALUE, 1, 1, false, Long.MIN_VALUE, Long.MAX_VALUE,
+                            sequenceTimestamp);
                     long[] seqValues = new long[1];
                     SQLException[] sqlExceptions = new SQLException[1];
                     long timestamp = scn == null ? HConstants.LATEST_TIMESTAMP : scn;
@@ -1465,13 +1465,13 @@ public class MetaDataClient {
 
         if (logger.isInfoEnabled()) logger.info("Created index " + table.getName().getString() + " at " + table.getTimeStamp());
         boolean asyncIndexBuildEnabled = connection.getQueryServices().getProps().getBoolean(
-            QueryServices.INDEX_ASYNC_BUILD_ENABLED,
-            QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
+                QueryServices.INDEX_ASYNC_BUILD_ENABLED,
+                QueryServicesOptions.DEFAULT_INDEX_ASYNC_BUILD_ENABLED);
         // In async process, we return immediately as the MR job needs to be triggered .
         if(statement.isAsync() && asyncIndexBuildEnabled) {
             return new MutationState(0, connection);
         }
-        
+
         // If our connection is at a fixed point-in-time, we need to open a new
         // connection so that our new index table is visible.
         if (connection.getSCN() != null) {
@@ -1514,7 +1514,7 @@ public class MetaDataClient {
         }
         return createSequence(tenantId, schemaName, statement
                 .getSequenceName().getTableName(), statement.ifNotExists(), startWith, incrementBy,
-            cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
+                cacheSize, statement.getCycle(), minValue, maxValue, timestamp);
     }
 
     private MutationState createSequence(String tenantId, String schemaName, String sequenceName,
@@ -1522,7 +1522,7 @@ public class MetaDataClient {
             long minValue, long maxValue, long timestamp) throws SQLException {
         try {
             connection.getQueryServices().createSequence(tenantId, schemaName, sequenceName,
-                startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
+                    startWith, incrementBy, cacheSize, minValue, maxValue, cycle, timestamp);
         } catch (SequenceAlreadyExistsException e) {
             if (ifNotExists) {
                 return new MutationState(0, connection);
@@ -1568,23 +1568,23 @@ public class MetaDataClient {
             case FUNCTION_ALREADY_EXISTS:
                 if (!function.isReplace()) {
                     throw new FunctionAlreadyExistsException(function.getFunctionName(), result
-                        .getFunctions().get(0));
+                            .getFunctions().get(0));
                 } else {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                     addFunctionToCache(result);
                 }
             case NEWER_FUNCTION_FOUND:
-                    // Add function to ConnectionQueryServices so it's cached, but don't add
-                    // it to this connection as we can't see it.
-                    throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
+                // Add function to ConnectionQueryServices so it's cached, but don't add
+                // it to this connection as we can't see it.
+                throw new NewerFunctionAlreadyExistsException(function.getFunctionName(), result.getFunctions().get(0));
             default:
                 List<PFunction> functions = new ArrayList<PFunction>(1);
                 functions.add(function);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), functions, true);
                 if(function.isReplace()) {
                     connection.removeFunction(function.getTenantId(), function.getFunctionName(),
-                        result.getMutationTime());
+                            result.getMutationTime());
                 }
                 addFunctionToCache(result);
             }
@@ -1593,7 +1593,7 @@ public class MetaDataClient {
         }
         return new MutationState(1, connection);
     }
-    
+
     private static ColumnDef findColumnDefOrNull(List<ColumnDef> colDefs, ColumnName colName) {
         for (ColumnDef colDef : colDefs) {
             if (colDef.getColumnDefName().getColumnName().equals(colName.getColumnName())) {
@@ -1602,7 +1602,7 @@ public class MetaDataClient {
         }
         return null;
     }
-    
+
     private static boolean checkAndValidateRowTimestampCol(ColumnDef colDef, PrimaryKeyConstraint pkConstraint,
             boolean rowTimeStampColAlreadyFound, PTableType tableType) throws SQLException {
 
@@ -1620,16 +1620,16 @@ public class MetaDataClient {
             if (isColumnDeclaredRowTimestamp) {
                 boolean isColumnPartOfPk = colDef.isPK() || pkConstraint.contains(columnDefName);
                 // A column can be declared as ROW_TIMESTAMP only if it is part of the primary key
-                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) { 
+                if (isColumnDeclaredRowTimestamp && !isColumnPartOfPk) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_PK_COL_ONLY)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // A column can be declared as ROW_TIMESTAMP only if it can be represented as a long
                 PDataType dataType = colDef.getDataType();
-                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) { 
+                if (isColumnDeclaredRowTimestamp && (dataType != PLong.INSTANCE && dataType != PUnsignedLong.INSTANCE && !dataType.isCoercibleTo(PTimestamp.INSTANCE))) {
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_COL_INVALID_TYPE)
-                    .setColumnName(columnDefName.getColumnName()).build().buildException(); 
+                    .setColumnName(columnDefName.getColumnName()).build().buildException();
                 }
 
                 // Only one column can be declared as a ROW_TIMESTAMP column
@@ -1642,7 +1642,7 @@ public class MetaDataClient {
         }
         return false;
     }
-    
+
     private PTable createTableInternal(CreateTableStatement statement, byte[][] splits,
             final PTable parent, String viewStatement, ViewType viewType,
             final byte[][] viewColumnConstants, final BitSet isViewColumnReferenced, Short indexId,
@@ -1755,8 +1755,8 @@ public class MetaDataClient {
                     isImmutableRows = isImmutableRowsProp;
                 }
             }
-            
-            if (tableType == PTableType.TABLE) { 
+
+            if (tableType == PTableType.TABLE) {
                 Boolean isAppendOnlySchemaProp = (Boolean) TableProperty.APPEND_ONLY_SCHEMA.getValue(tableProps);
                 isAppendOnlySchema = isAppendOnlySchemaProp!=null ? isAppendOnlySchemaProp : false;
             }
@@ -1779,7 +1779,7 @@ public class MetaDataClient {
                 }
                 addSaltColumn = (saltBucketNum != null);
             }
-            
+
             // Can't set MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an INDEX or a non mapped VIEW
             if (tableType != PTableType.INDEX && (tableType != PTableType.VIEW || viewType == ViewType.MAPPED)) {
                 Boolean multiTenantProp = (Boolean) tableProps.get(PhoenixDatabaseMetaData.MULTI_TENANT);
@@ -1840,7 +1840,7 @@ public class MetaDataClient {
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
             }
-            
+
             // Put potentially inferred value into tableProps as it's used by the createTable call below
             // to determine which coprocessors to install on the new table.
             tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, transactional);
@@ -1851,9 +1851,9 @@ public class MetaDataClient {
                     commonFamilyProps.put(TxConstants.PROPERTY_TTL, ttl);
                 }
             }
-            
+
             boolean sharedTable = statement.getTableType() == PTableType.VIEW || indexId != null;
-            if (transactional) { 
+            if (transactional) {
                 // Tephra uses an empty value cell as its delete marker, so we need to turn on
                 // storeNulls for transactional tables.
                 // If we use regular column delete markers (which is what non transactional tables
@@ -1869,7 +1869,7 @@ public class MetaDataClient {
                 // Force STORE_NULLS to true when transactional as Tephra cannot deal with column deletes
                 storeNulls = true;
                 tableProps.put(PhoenixDatabaseMetaData.STORE_NULLS, Boolean.TRUE);
-                
+
                 if (!sharedTable) {
                     Integer maxVersionsProp = (Integer) commonFamilyProps.get(HConstants.VERSIONS);
                     if (maxVersionsProp == null) {
@@ -2010,7 +2010,7 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-            
+
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2156,20 +2156,20 @@ public class MetaDataClient {
             }
 
             short nextKeySeq = 0;
-            
+
             List<Mutation> columnMetadata = Lists.newArrayListWithExpectedSize(columns.size());
             try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_CREATE_TABLE)) {
                 for (Map.Entry<PColumn, PColumn> entry : columns.entrySet()) {
                     PColumn column = entry.getValue();
                     final int columnPosition = column.getPosition();
                     // For client-side cache, we need to update the column
-                    // set the autoPartition column attributes   
+                    // set the autoPartition column attributes
                     if (parent != null && parent.getAutoPartitionSeqName() != null
                             && parent.getPKColumns().get(MetaDataUtil.getAutoPartitionColIndex(parent)).equals(column)) {
                         entry.setValue(column = new DelegateColumn(column) {
                             @Override
                             public byte[] getViewConstant() {
-                                // set to non-null value so that we will generate a Put that 
+                                // set to non-null value so that we will generate a Put that
                                 // will be set correctly on the server
                                 return QueryConstants.EMPTY_COLUMN_VALUE_BYTES;
                             }
@@ -2230,7 +2230,7 @@ public class MetaDataClient {
             tableUpsert.setBoolean(11, isImmutableRows);
             tableUpsert.setString(12, defaultFamilyName);
             if (parent != null && parent.getAutoPartitionSeqName() != null && viewStatement==null) {
-                // set to non-null value so that we will generate a Put that 
+                // set to non-null value so that we will generate a Put that
                 // will be set correctly on the server
                 tableUpsert.setString(13, QueryConstants.EMPTY_COLUMN_VALUE);
             }
@@ -2428,7 +2428,7 @@ public class MetaDataClient {
         return dropTable(schemaName, tableName, parentTableName, PTableType.INDEX, statement.ifExists(), false);
     }
 
-    private MutationState dropFunction(String functionName, 
+    private MutationState dropFunction(String functionName,
             boolean ifExists) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
@@ -2446,7 +2446,7 @@ public class MetaDataClient {
                     return new MutationState(0, connection);
                 }
             } catch(FunctionNotFoundException e) {
-                
+
             }
             List<Mutation> functionMetaData = Lists.newArrayListWithExpectedSize(2);
             Delete functionDelete = new Delete(key, clientTimeStamp);
@@ -2523,7 +2523,7 @@ public class MetaDataClient {
                         // All multi-tenant tables have a view index table, so no need to check in that case
                         if (parentTableName == null) {
                             hasViewIndexTable = true;// keeping always true for deletion of stats if view index present
-                                                     // or not
+                            // or not
                             MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(),
                                     table.isNamespaceMapped());
                             byte[] viewIndexPhysicalName = MetaDataUtil
@@ -2635,7 +2635,7 @@ public class MetaDataClient {
                 msg = "Cannot add/drop column referenced by VIEW";
             }
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-                .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).setFamilyName(familyName).setColumnName(columnName).setMessage(msg).build().buildException();
         case NO_OP:
         case COLUMN_ALREADY_EXISTS:
         case COLUMN_NOT_FOUND:
@@ -2648,18 +2648,18 @@ public class MetaDataClient {
             throw new ConcurrentTableMutationException(schemaName, tableName);
         case NEWER_TABLE_FOUND:
             // TODO: update cache?
-//            if (result.getTable() != null) {
-//                connection.addTable(result.getTable());
-//            }
+            //            if (result.getTable() != null) {
+            //                connection.addTable(result.getTable());
+            //            }
             throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
         case NO_PK_COLUMNS:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            .setSchemaName(schemaName).setTableName(tableName).build().buildException();
         case TABLE_ALREADY_EXISTS:
             break;
         default:
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNEXPECTED_MUTATION_CODE).setSchemaName(schemaName)
-                .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
+            .setTableName(tableName).setMessage("mutation code: " + mutationCode).build().buildException();
         }
         return mutationCode;
     }
@@ -2713,11 +2713,11 @@ public class MetaDataClient {
     private void mutateBooleanProperty(String tenantId, String schemaName, String tableName,
             String propertyName, boolean propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2730,11 +2730,11 @@ public class MetaDataClient {
     private void mutateLongProperty(String tenantId, String schemaName, String tableName,
             String propertyName, long propertyValue) throws SQLException {
         String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
-                        TENANT_ID + "," +
-                        TABLE_SCHEM + "," +
-                        TABLE_NAME + "," +
-                        propertyName +
-                        ") VALUES (?, ?, ?, ?)";
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
         try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
             tableBoolUpsert.setString(1, tenantId);
             tableBoolUpsert.setString(2, schemaName);
@@ -2743,7 +2743,7 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
-    
+
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
         return addColumn(table, statement.getColumnDefs(), statement.getProps(), statement.ifNotExists(), false, statement.getTable(), statement.getTableType());
@@ -2752,7 +2752,7 @@ public class MetaDataClient {
     public MutationState addColumn(PTable table, List<ColumnDef> origColumnDefs,
             ListMultimap<String, Pair<String, Object>> stmtProperties, boolean ifNotExists,
             boolean removeTableProps, NamedTableNode namedTableNode, PTableType tableType)
-            throws SQLException {
+                    throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
@@ -3029,12 +3029,12 @@ public class MetaDataClient {
                     connection.rollback();
                 }
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) { 
+                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
                     seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                
+
                 // Force the table header row to be first
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
@@ -3069,7 +3069,7 @@ public class MetaDataClient {
                         return new MutationState(0,connection);
                     }
 
-                    // Only update client side cache if we aren't adding a PK column to a table with indexes or 
+                    // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
@@ -3082,13 +3082,13 @@ public class MetaDataClient {
                                 result.getMutationTime(),
                                 seqNum,
                                 isImmutableRows == null ? table.isImmutableRows() : isImmutableRows,
-                                disableWAL == null ? table.isWALDisabled() : disableWAL,
-                                multiTenant == null ? table.isMultiTenant() : multiTenant,
-                                storeNulls == null ? table.getStoreNulls() : storeNulls, 
-                                isTransactional == null ? table.isTransactional() : isTransactional,
-                                updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
-                                table.isNamespaceMapped(),
-                                resolvedTimeStamp);
+                                        disableWAL == null ? table.isWALDisabled() : disableWAL,
+                                                multiTenant == null ? table.isMultiTenant() : multiTenant,
+                                                        storeNulls == null ? table.getStoreNulls() : storeNulls,
+                                                                isTransactional == null ? table.isTransactional() : isTransactional,
+                                                                        updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
+                                                                                table.isNamespaceMapped(),
+                                                                                resolvedTimeStamp);
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3172,7 +3172,7 @@ public class MetaDataClient {
         Collections.sort(columnsToDrop,new Comparator<PColumn> () {
             @Override
             public int compare(PColumn left, PColumn right) {
-               return Ints.compare(left.getPosition(), right.getPosition());
+                return Ints.compare(left.getPosition(), right.getPosition());
             }
         });
 
@@ -3194,7 +3194,7 @@ public class MetaDataClient {
             colUpdate.setInt(6, column.getPosition() - columnsToDropIndex - (isSalted ? 1 : 0));
             colUpdate.execute();
         }
-       return familyName;
+        return familyName;
     }
 
     /**
@@ -3227,7 +3227,7 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
-                
+
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
                     columnRefs = Lists.newArrayListWithCapacity(0);
@@ -3251,7 +3251,7 @@ public class MetaDataClient {
                     tableColumnsToDrop.add(columnToDrop);
                     if (SchemaUtil.isPKColumn(columnToDrop)) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_PK)
-                            .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                        .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
                     else if (table.isAppendOnlySchema()) {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
@@ -3267,18 +3267,18 @@ public class MetaDataClient {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
                     // get the columns required for the index pk
                     Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns 
+                    // get the covered columns
                     Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
                     for(PColumn columnToDrop : tableColumnsToDrop) {
                         ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
                         // if the columns being dropped is indexed and the physical index table is not shared
                         if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null) 
+                            if (index.getViewIndexId()==null)
                                 indexesToDrop.add(new TableRef(index));
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
-                        } 
+                        }
                         else if (coveredColumns.contains(columnToDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
                             PColumn indexColumn = index.getColumn(indexColumnName);
@@ -3293,8 +3293,8 @@ public class MetaDataClient {
                         dropColumnMutations(index, indexColumnsToDrop);
                         long clientTimestamp = MutationState.getMutationTimestamp(timeStamp, connection.getSCN());
                         connection.removeColumn(tenantId, index.getName().getString(),
-                            indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
-                            TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
+                                indexColumnsToDrop, clientTimestamp, indexTableSeqNum,
+                                TransactionUtil.getResolvedTimestamp(connection, index.isTransactional(), clientTimestamp));
                     }
                 }
                 tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
@@ -3332,8 +3332,8 @@ public class MetaDataClient {
                             connection.getQueryServices().addColumn(
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
-                                            tableContainingColumnToDrop.getTableName().getBytes()))),
-                                            tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
+                                                    tableContainingColumnToDrop.getTableName().getBytes()))),
+                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)));
 
                         }
                     }
@@ -3354,7 +3354,7 @@ public class MetaDataClient {
                     if (tableColumnsToDrop.size() > 0) {
                         if (removedIndexTableOrColumn)
                             connection.removeTable(tenantId, tableName, table.getParentName() == null ? null : table.getParentName().getString(), table.getTimeStamp());
-                        else  
+                        else
                             connection.removeColumn(tenantId, SchemaUtil.getTableName(schemaName, tableName) , tableColumnsToDrop, result.getMutationTime(), seqNum, TransactionUtil.getResolvedTime(connection, result));
                     }
                     // If we have a VIEW, then only delete the metadata, and leave the table data alone
@@ -3365,10 +3365,10 @@ public class MetaDataClient {
                         // Delete everything in the column. You'll still be able to do queries at earlier timestamps
                         long ts = (scn == null ? result.getMutationTime() : scn);
                         PostDDLCompiler compiler = new PostDDLCompiler(connection);
-                        
+
                         boolean dropMetaData = connection.getQueryServices().getProps().getBoolean(DROP_METADATA_ATTRIB, DEFAULT_DROP_METADATA);
                         // if the index is a local index or view index it uses a shared physical table
-                        // so we need to issue deletes markers for all the rows of the index 
+                        // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
                         if (result.getSharedTablesToDelete()!=null) {
@@ -3389,7 +3389,7 @@ public class MetaDataClient {
                                     }
                                     tenantIdTableRefMap.get(indexTableTenantId.getString()).add(indexTableRef);
                                 }
-                                
+
                             }
                         }
                         // if dropMetaData is false delete all rows for the indexes (if it was true
@@ -3399,7 +3399,7 @@ public class MetaDataClient {
                         }
                         // Drop any index tables that had the dropped column in the PK
                         connection.getQueryServices().updateData(compiler.compile(tableRefsToDrop, null, null, Collections.<PColumn>emptyList(), ts));
-                        
+
                         // Drop any tenant-specific indexes
                         if (!tenantIdTableRefMap.isEmpty()) {
                             for (Entry<String, List<TableRef>> entry : tenantIdTableRefMap.entrySet()) {
@@ -3412,7 +3412,7 @@ public class MetaDataClient {
                                 }
                             }
                         }
-                        
+
                         // Update empty key value column if necessary
                         for (ColumnRef droppedColumnRef : columnsToDrop) {
                             // Painful, but we need a TableRef with a pre-set timestamp to prevent attempts
@@ -3572,21 +3572,21 @@ public class MetaDataClient {
     }
 
     private void throwIfLastPKOfParentIsFixedLength(PTable parent, String viewSchemaName, String viewName, ColumnDef col) throws SQLException {
-        if (isLastPKVariableLength(parent)) { 
+        if (isLastPKVariableLength(parent)) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MODIFY_VIEW_PK)
-                .setSchemaName(viewSchemaName)
-                .setTableName(viewName)
-                .setColumnName(col.getColumnDefName().getColumnName())
-                .build().buildException(); }
+            .setSchemaName(viewSchemaName)
+            .setTableName(viewName)
+            .setColumnName(col.getColumnDefName().getColumnName())
+            .build().buildException(); }
     }
-    
+
     private boolean isLastPKVariableLength(PTable table) {
         List<PColumn> pkColumns = table.getPKColumns();
         return !pkColumns.get(pkColumns.size()-1).getDataType().isFixedWidth();
     }
-    
+
     private PTable getParentOfView(PTable view) throws SQLException {
-        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114 
+        //TODO just use view.getParentName().getString() after implementing https://issues.apache.org/jira/browse/PHOENIX-2114
         SelectStatement select = new SQLParser(view.getViewStatement()).parseQuery();
         String parentName = SchemaUtil.normalizeFullTableName(select.getFrom().toString().trim());
         return connection.getTable(new PTableKey(view.getTenantId(), parentName));
@@ -3598,9 +3598,9 @@ public class MetaDataClient {
         try {
             if (!SchemaUtil.isNamespaceMappingEnabled(null,
                     connection.getQueryServices()
-                            .getProps())) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
-                                            .build().buildException(); }
+                    .getProps())) { throw new SQLExceptionInfo.Builder(
+                            SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
+                            .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
             validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
@@ -3639,7 +3639,7 @@ public class MetaDataClient {
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
                 schemaName.toUpperCase())) { throw new SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
-                        .setSchemaName(schemaName).build().buildException(); }
+                .setSchemaName(schemaName).build().buildException(); }
     }
 
     public MutationState dropSchema(DropSchemaStatement executableDropSchemaStatement) throws SQLException {
@@ -3667,7 +3667,7 @@ public class MetaDataClient {
                 throw new NewerSchemaAlreadyExistsException(schemaName);
             case TABLES_EXIST_ON_SCHEMA:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_SCHEMA).setSchemaName(schemaName)
-                        .build().buildException();
+                .build().buildException();
             default:
                 connection.removeSchema(schema, result.getMutationTime());
                 break;
@@ -3685,7 +3685,7 @@ public class MetaDataClient {
             connection.setSchema(null);
         } else {
             FromCompiler.getResolverForSchema(useSchemaStatement, connection)
-                    .resolveSchema(useSchemaStatement.getSchemaName());
+            .resolveSchema(useSchemaStatement.getSchemaName());
             connection.setSchema(useSchemaStatement.getSchemaName());
         }
         return new MutationState(0, connection);


[18/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

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

Branch: refs/heads/4.8-HBase-1.0
Commit: ccd04ba6862903e5d2975c8be74239eafc40dbad
Parents: 56ea4ae
Author: James Taylor <ja...@apache.org>
Authored: Tue Sep 13 22:37:18 2016 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Wed Sep 14 17:25:14 2016 -0400

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/IndexIT.java   | 1172 ++++++++++--------
 .../query/ConnectionQueryServicesImpl.java      | 1009 +++++++--------
 .../apache/phoenix/query/QueryConstants.java    |    5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  568 ++++-----
 4 files changed, 1434 insertions(+), 1320 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ccd04ba6/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index b7537a6..41d2f60 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -22,6 +22,7 @@ import static org.apache.phoenix.query.QueryConstants.MILLIS_IN_DAY;
 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.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -45,10 +46,12 @@ import org.apache.hadoop.hbase.CellScanner;
 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.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.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
@@ -59,6 +62,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
@@ -79,162 +83,162 @@ import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
 public class IndexIT extends BaseHBaseManagedTimeIT {
-	
-	private final boolean localIndex;
+    
+    private final boolean localIndex;
     private final boolean transactional;
     private final boolean mutable;
-	private final String tableDDLOptions;
-	private final String tableName;
+    private final String tableDDLOptions;
+    private final String tableName;
     private final String indexName;
     private final String fullTableName;
     private final String fullIndexName;
-	
-	public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
-		this.localIndex = localIndex;
-		this.transactional = transactional;
-		this.mutable = mutable;
-		StringBuilder optionBuilder = new StringBuilder();
-		if (!mutable) 
-			optionBuilder.append(" IMMUTABLE_ROWS=true ");
-		if (transactional) {
-			if (!(optionBuilder.length()==0))
-				optionBuilder.append(",");
-			optionBuilder.append(" TRANSACTIONAL=true ");
-		}
-		this.tableDDLOptions = optionBuilder.toString();
-		this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
+    
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+        this.localIndex = localIndex;
+        this.transactional = transactional;
+        this.mutable = mutable;
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!mutable) 
+            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (transactional) {
+            if (!(optionBuilder.length()==0))
+                optionBuilder.append(",");
+            optionBuilder.append(" TRANSACTIONAL=true ");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+        this.tableName = TestUtil.DEFAULT_DATA_TABLE_NAME + ( transactional ?  "_TXN" : "");
         this.indexName = "IDX" + ( transactional ?  "_TXN" : "");
         this.fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         this.fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	}
-	
-	@BeforeClass
+    }
+    
+    @BeforeClass
     @Shadower(classBeingShadowed = BaseHBaseManagedTimeIT.class)
     public static void doSetup() throws Exception {
         Map<String,String> props = Maps.newHashMapWithExpectedSize(1);
         props.put(QueryServices.TRANSACTIONS_ENABLED, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
-	
-	@Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
+
+    @Parameters(name="localIndex = {0} , mutable = {1} , transactional = {2}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {     
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true }, 
+        return Arrays.asList(new Boolean[][] {
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
                  { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
 
-	@Test
+    @Test
     public void testIndexWithNullableFixedWithCols() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName 
-	                    + " (char_col1 ASC, int_col1 ASC)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
-	        ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex) {
-	            assertEquals(
-	                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
-	                    + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
-	        }
-	        
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals("chara", rs.getString("char_col1"));
-	        assertEquals(2, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(3, rs.getInt(2));
-	        assertTrue(rs.next());
-	        assertEquals("chara", rs.getString(1));
-	        assertEquals(4, rs.getInt(2));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
-	        
-	        query = "SELECT char_col1, int_col1 from " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        
-	        query = "SELECT char_col1, int_col1 from "+indexName;
-	        try{
-	            rs = conn.createStatement().executeQuery(query);
-	            fail();
-	        } catch (SQLException e) {
-	            assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
-	        }
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                    + " (char_col1 ASC, int_col1 ASC)"
+                    + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            String query = "SELECT d.char_col1, int_col1 from " + fullTableName + " as d";
+            ResultSet rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex) {
+                assertEquals(
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + tableName + " [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"
+                        + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals("chara", rs.getString("char_col1"));
+            assertEquals(2, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(3, rs.getInt(2));
+            assertTrue(rs.next());
+            assertEquals("chara", rs.getString(1));
+            assertEquals(4, rs.getInt(2));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+
+            query = "SELECT char_col1, int_col1 from " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+
+            query = "SELECT char_col1, int_col1 from "+indexName;
+            try{
+                rs = conn.createStatement().executeQuery(query);
+                fail();
+            } catch (SQLException e) {
+                assertEquals(SQLExceptionCode.TABLE_UNDEFINED.getErrorCode(), e.getErrorCode());
+            }
         }
     }
-    
+
     @Test
     public void testDeleteFromAllPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_pk, varchar_pk)"
-	                    + " INCLUDE (long_col1, long_col2)";
-	        stmt.execute(ddl);
-	        
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-	        
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_pk, varchar_pk)"
+                        + " INCLUDE (long_col1, long_col2)";
+            stmt.execute(ddl);
+
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testCreateIndexAfterUpsertStarted() throws Exception {
         testCreateIndexAfterUpsertStarted(false, fullTableName + "1", fullIndexName + "1");
@@ -253,13 +257,13 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             BaseTest.populateTestTable(fullTableName);
 
             ResultSet rs;
-            
+
             rs = conn1.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
             assertTrue(rs.next());
             assertEquals(3,rs.getInt(1));
 
             try (Connection conn2 = DriverManager.getConnection(getUrl(), props)) {
-                
+
                 String upsert = "UPSERT INTO " + fullTableName
                         + " VALUES(?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
                 PreparedStatement pstmt2 = conn2.prepareStatement(upsert);
@@ -283,39 +287,39 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 pstmt2.setBigDecimal(17, new BigDecimal(3.0));
                 pstmt2.setDate(18, date);
                 pstmt2.executeUpdate();
-                
+
                 if (readOwnWrites) {
                     String query = "SELECT long_pk FROM " + fullTableName + " WHERE long_pk=4";
                     rs = conn2.createStatement().executeQuery(query);
                     assertTrue(rs.next());
                     assertFalse(rs.next());
                 }
-                
+
                 String indexName = SchemaUtil.getTableNameFromFullName(fullIndexName);
                 ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
                         + " (long_pk, varchar_pk)"
                         + " INCLUDE (long_col1, long_col2)";
                 stmt1.execute(ddl);
-                
+
                 /*
                  * Commit upsert after index created through different connection.
                  * This forces conn2 (which doesn't know about the index yet) to update the metadata
                  * at commit time, recognize the new index, and generate the correct metadata (or index
                  * rows for immutable indexes).
-                 * 
+                 *
                  * For transactional data, this is problematic because the index
                  * gets a timestamp *after* the commit timestamp of conn2 and thus won't be seen during
                  * the commit. Also, when the index is being built, the data hasn't yet been committed
                  * and thus won't be part of the initial index build (fixed by PHOENIX-2446).
                  */
                 conn2.commit();
-                
+
                 stmt1 = conn1.createStatement();
                 rs = stmt1.executeQuery("SELECT COUNT(*) FROM " + fullTableName);
                 assertTrue(rs.next());
                 assertEquals(4,rs.getInt(1));
                 assertEquals(fullIndexName, stmt1.unwrap(PhoenixStatement.class).getQueryPlan().getTableRef().getTable().getName().getString());
-                
+
                 String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
                 rs = conn1.createStatement().executeQuery(query);
                 assertTrue(rs.next());
@@ -330,85 +334,91 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             }
         }
     }
-    
+
     @Test
     public void testDeleteFromNonPKColumnIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
-	                    + " (long_col1, long_col2)"
-	                    + " INCLUDE (decimal_col1, decimal_col2)";
-	        stmt.execute(ddl);
+            conn.setAutoCommit(false);
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName
+                        + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+            stmt.execute(ddl);
         }
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {    
-	        ResultSet rs;
-	        
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
-	        assertTrue(rs.next());
-	        assertEquals(3,rs.getInt(1));
-	        
-	        String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
-	        assertEquals(1,conn.createStatement().executeUpdate(dml));
-	        conn.commit();
-
-	        // query the data table
-	        String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        // query the index table
-	        query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals(1L, rs.getLong(1));
-	        assertTrue(rs.next());
-	        assertEquals(3L, rs.getLong(1));
-	        assertFalse(rs.next());
-	        
-	        conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            ResultSet rs;
+
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullTableName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+            rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexName);
+            assertTrue(rs.next());
+            assertEquals(3,rs.getInt(1));
+
+            String dml = "DELETE from " + fullTableName + " WHERE long_col2 = 4";
+            assertEquals(1,conn.createStatement().executeUpdate(dml));
+            conn.commit();
+
+            // query the data table
+            String query = "SELECT /*+ NO_INDEX */ long_pk FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            // query the index table
+            query = "SELECT long_pk FROM " + fullTableName + " ORDER BY long_col1";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals(1L, rs.getLong(1));
+            assertTrue(rs.next());
+            assertEquals(3L, rs.getLong(1));
+            assertFalse(rs.next());
+
+            conn.createStatement().execute("DROP INDEX " + indexName + " ON " + fullTableName);
         }
     }
-    
+
     @Test
     public void testGroupByCount() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
-	        stmt.execute(ddl);
-	        ResultSet rs;
-	        rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
-	        assertTrue(rs.next());
-	        assertEquals(1,rs.getInt(2));
-    	}
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+            stmt.execute(ddl);
+            ResultSet rs;
+            rs = conn.createStatement().executeQuery("SELECT int_col2, COUNT(*) FROM " + fullTableName + " GROUP BY int_col2");
+            assertTrue(rs.next());
+            assertEquals(1,rs.getInt(2));
+        }
     }
 
     @Test
     public void testSelectDistinctOnTableWithSecondaryImmutableIndex() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col2)";
             PreparedStatement pstmt = conn.prepareStatement(ddl);
             pstmt.execute();
             ResultSet rs = conn.createStatement().executeQuery("SELECT distinct int_col2 FROM " + fullTableName + " where int_col2 > 0");
@@ -419,19 +429,22 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(5, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
 
     @Test
     public void testInClauseWithIndexOnColumnOfUsignedIntType() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-	        BaseTest.populateTestTable(fullTableName);
-	        ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.setAutoCommit(false);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+            BaseTest.populateTestTable(fullTableName);
+            ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName + " ON " + fullTableName + " (int_col1)";
             stmt.execute(ddl);
             ResultSet rs = conn.createStatement().executeQuery("SELECT int_col1 FROM " + fullTableName + " where int_col1 IN (1, 2, 3, 4)");
             assertTrue(rs.next());
@@ -441,72 +454,77 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(4, rs.getInt(1));
             assertFalse(rs.next());
-    	}
+        }
     }
-    
+
     @Test
     public void createIndexOnTableWithSpecifiedDefaultCF() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
+            conn.setAutoCommit(false);
+            String query;
             ResultSet rs;
-	        String ddl ="CREATE TABLE " + fullTableName 
-	        		+ " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
-	        Statement stmt = conn.createStatement();
-	        stmt.execute(ddl);
-
-	        query = "SELECT * FROM " + tableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-
-	        String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
-	        conn.createStatement().execute(
-	                "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
-	        query = "SELECT * FROM " + fullIndexName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        //check options set correctly on index
-	        TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
-	        NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
-	        ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
-	        PTable indexTable = resolver.getTables().get(0).getTable();
-	        // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
-	        assertNull(indexTable.getDefaultFamilyName());
-	        assertFalse(indexTable.isMultiTenant());
-	        assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
-	        if(localIndex) {
-	            assertEquals(10, indexTable.getBucketNum().intValue());
-	            assertTrue(indexTable.isWALDisabled());
-	        }
+            String ddl ="CREATE TABLE " + fullTableName
+                    + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) DEFAULT_COLUMN_FAMILY='A'" + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "");
+            Statement stmt = conn.createStatement();
+            stmt.execute(ddl);
+
+            query = "SELECT * FROM " + tableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            String options = localIndex ? "SALT_BUCKETS=10, MULTI_TENANT=true, IMMUTABLE_ROWS=true, DISABLE_WAL=true" : "";
+            conn.createStatement().execute(
+                    "CREATE INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE (v2) " + options);
+            query = "SELECT * FROM " + fullIndexName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            //check options set correctly on index
+            TableName indexTableName = TableName.create(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+            NamedTableNode indexNode = NamedTableNode.create(null, indexTableName, null);
+            ColumnResolver resolver = FromCompiler.getResolver(indexNode, conn.unwrap(PhoenixConnection.class));
+            PTable indexTable = resolver.getTables().get(0).getTable();
+            // Can't set IMMUTABLE_ROWS, MULTI_TENANT or DEFAULT_COLUMN_FAMILY_NAME on an index
+            assertNull(indexTable.getDefaultFamilyName());
+            assertFalse(indexTable.isMultiTenant());
+            assertEquals(mutable, !indexTable.isImmutableRows()); // Should match table
+            if(localIndex) {
+                assertEquals(10, indexTable.getBucketNum().intValue());
+                assertTrue(indexTable.isWALDisabled());
+            }
         }
     }
-    
+
     @Test
     public void testIndexWithNullableDateCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        String fullIndexName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, indexName);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
+            conn.setAutoCommit(false);
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
+
             populateMultiCFTestTable(fullTableName, date);
             String ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (date_col)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             String query = "SELECT int_pk from " + fullTableName ;
             ResultSet 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));
+                        + "    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));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
@@ -515,7 +533,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
             assertFalse(rs.next());
-            
+
             query = "SELECT date_col from " + fullTableName + " order by date_col" ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -526,7 +544,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName + "\n"
                         + "    SERVER FILTER BY FIRST KEY ONLY", QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(date, rs.getDate(1));
@@ -535,303 +553,315 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertTrue(rs.next());
             assertEquals(new Date(date.getTime() + 2 * MILLIS_IN_DAY), rs.getDate(1));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
+
     @Test
     public void testSelectAllAndAliasWithIndex() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
-	        conn.createStatement().execute(ddl);
-	        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();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        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("b",rs.getString(1));
-	        assertEquals("y",rs.getString(2));
-	        assertEquals("2",rs.getString(3));
-	        assertEquals("b",rs.getString("k"));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("a",rs.getString(1));
-	        assertEquals("x",rs.getString(2));
-	        assertEquals("1",rs.getString(3));
-	        assertEquals("a",rs.getString("k"));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
-	        
-	        query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if(localIndex){
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        } else {
-	            assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" + 
-	                    "    SERVER SORTED BY [\"V1\"]\n" + 
-	                    "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
-	        }
-
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("x",rs.getString("foo"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (v1)";
+            conn.createStatement().execute(ddl);
+            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();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            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("b",rs.getString(1));
+            assertEquals("y",rs.getString(2));
+            assertEquals("2",rs.getString(3));
+            assertEquals("b",rs.getString("k"));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("a",rs.getString(1));
+            assertEquals("x",rs.getString(2));
+            assertEquals("1",rs.getString(3));
+            assertEquals("a",rs.getString("k"));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
+
+            query = "SELECT v1 as foo FROM " + fullTableName + " WHERE v2 = '1' ORDER BY foo";
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if(localIndex){
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullTableName + " [1,~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            } else {
+                assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " +fullIndexName + " [~'1']\n" +
+                        "    SERVER SORTED BY [\"V1\"]\n" +
+                        "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+            }
+
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("x",rs.getString("foo"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testSelectCF() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
-			conn.createStatement().execute(ddl);
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	        ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
-	        conn.createStatement().execute(ddl);
-	        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.setString(4, "A");
-	        stmt.execute();
-	        stmt.setString(1,"b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.setString(4, "B");
-	        stmt.execute();
-	        conn.commit();
-	        
-	        query = "SELECT * FROM " + fullTableName;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
-	
-	        query = "SELECT a.* 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("y",rs.getString(1));
-	        assertEquals("2",rs.getString(2));
-	        assertEquals("y",rs.getString("v1"));
-	        assertEquals("2",rs.getString("v2"));
-	        assertTrue(rs.next());
-	        assertEquals("x",rs.getString(1));
-	        assertEquals("1",rs.getString(2));
-	        assertEquals("x",rs.getString("v1"));
-	        assertEquals("1",rs.getString("v2"));
-	        assertFalse(rs.next());
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName + " (k VARCHAR NOT NULL PRIMARY KEY, a.v1 VARCHAR, a.v2 VARCHAR, b.v1 VARCHAR) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+            ddl = "CREATE " + (localIndex ? " LOCAL " : "") + " INDEX " + indexName + " ON " + fullTableName + " (v2 DESC) INCLUDE (a.v1)";
+            conn.createStatement().execute(ddl);
+            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.setString(4, "A");
+            stmt.execute();
+            stmt.setString(1,"b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.setString(4, "B");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullTableName, QueryUtil.getExplainPlan(rs));
+
+            query = "SELECT a.* 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("y",rs.getString(1));
+            assertEquals("2",rs.getString(2));
+            assertEquals("y",rs.getString("v1"));
+            assertEquals("2",rs.getString("v2"));
+            assertTrue(rs.next());
+            assertEquals("x",rs.getString(1));
+            assertEquals("1",rs.getString(2));
+            assertEquals("x",rs.getString("v1"));
+            assertEquals("1",rs.getString("v2"));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testUpsertAfterIndexDrop() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        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());
-	
-	        String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
-	        stmt = conn.prepareStatement(ddl);
-	        stmt.execute();
-	        
-	        stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "y");
-	        stmt.execute();
-	        conn.commit();
-	    
-	        query = "SELECT * FROM " + fullTableName;
-	    
-	        // 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));
-	        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());
+
+            String ddl = "DROP INDEX " + indexName + " ON " + fullTableName;
+            stmt = conn.prepareStatement(ddl);
+            stmt.execute();
+
+            stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + "(k, v1) VALUES(?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "y");
+            stmt.execute();
+            conn.commit();
+
+            query = "SELECT * FROM " + fullTableName;
+
+            // 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));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testMultipleUpdatesAcrossRegions() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-    	String testTable = fullTableName+"_MULTIPLE_UPDATES";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String testTable = fullTableName+"_MULTIPLE_UPDATES";
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + testTable
-	              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
-	        		  + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery(query);
-	        assertFalse(rs.next());
-	
-	        conn.createStatement().execute(
-	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
-	        stmt.setString(1, "a");
-	        stmt.setString(2, "x");
-	        stmt.setString(3, "1");
-	        stmt.execute();
-	        stmt.setString(1, "b");
-	        stmt.setString(2, "y");
-	        stmt.setString(3, "2");
-	        stmt.execute();
-	        stmt.setString(1, "c");
-	        stmt.setString(2, "z");
-	        stmt.setString(3, "3");
-	        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));
-	        assertTrue(rs.next());
-	        assertEquals("y", rs.getString(1));
-	        assertEquals("2", rs.getString(2));
-	        assertEquals("b", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("z", rs.getString(1));
-	        assertEquals("3", rs.getString(2));
-	        assertEquals("c", rs.getString(3));
-	        assertFalse(rs.next());
-	
-	        query = "SELECT * FROM " + testTable;
-	        rs = conn.createStatement().executeQuery("EXPLAIN " + query);
-	        if (localIndex) {
-	            assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
-	        assertEquals("1", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("b", rs.getString(1));
-	        assertEquals("y", rs.getString(2));
-	        assertEquals("2", rs.getString(3));
-	        assertTrue(rs.next());
-	        assertEquals("c", rs.getString(1));
-	        assertEquals("z", rs.getString(2));
-	        assertEquals("3", 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 " + testTable
+                      + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "  + HTableDescriptor.MAX_FILESIZE + "=1, " + HTableDescriptor.MEMSTORE_FLUSHSIZE + "=1 " 
+                      + (!tableDDLOptions.isEmpty() ? "," + tableDDLOptions : "") + "SPLIT ON ('b')");
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery(query);
+            assertFalse(rs.next());
+    
+            conn.createStatement().execute(
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + testTable + " (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 " + testTable + " VALUES(?,?,?)");
+            stmt.setString(1, "a");
+            stmt.setString(2, "x");
+            stmt.setString(3, "1");
+            stmt.execute();
+            stmt.setString(1, "b");
+            stmt.setString(2, "y");
+            stmt.setString(3, "2");
+            stmt.execute();
+            stmt.setString(1, "c");
+            stmt.setString(2, "z");
+            stmt.setString(3, "3");
+            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));
+            assertTrue(rs.next());
+            assertEquals("y", rs.getString(1));
+            assertEquals("2", rs.getString(2));
+            assertEquals("b", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("z", rs.getString(1));
+            assertEquals("3", rs.getString(2));
+            assertEquals("c", rs.getString(3));
+            assertFalse(rs.next());
+    
+            query = "SELECT * FROM " + testTable;
+            rs = conn.createStatement().executeQuery("EXPLAIN " + query);
+            if (localIndex) {
+                assertEquals("CLIENT PARALLEL 2-WAY RANGE SCAN OVER " + testTable+" [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("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
+            assertTrue(rs.next());
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
+            assertFalse(rs.next());
         }
     }
-    
+
     @Test
     public void testIndexWithCaseSensitiveCols() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            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);
             long ts = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,fullTableName)).getTimeStamp();
             assertFalse(rs.next());
             conn.createStatement().execute(
-  	  	          "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
+                    "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(\"v2\") INCLUDE (\"V1\")");
             query = "SELECT * FROM "+fullIndexName;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -851,7 +881,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1,'1']\n"
-                           + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
+                        + "CLIENT MERGE SORT", QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
@@ -870,7 +900,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex){
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullTableName + " [1]\nCLIENT MERGE SORT",
-                    QueryUtil.getExplainPlan(rs));
+                        QueryUtil.getExplainPlan(rs));
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER "+fullIndexName, QueryUtil.getExplainPlan(rs));
             }
@@ -899,9 +929,9 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals("2",rs.getString(5));
             assertEquals("2",rs.getString("v2"));
             assertFalse(rs.next());
-            
+
             assertNoIndexDeletes(conn, ts, fullIndexName);
-        } 
+        }
     }
 
     private void assertNoIndexDeletes(Connection conn, long minTimestamp, String fullIndexName) throws IOException, SQLException {
@@ -931,38 +961,45 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
 
     @Test
     public void testInFilterOnIndexedTable() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-	        conn.setAutoCommit(false);
-	        String query;
-	        ResultSet rs;
-	        String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
-	        conn.createStatement().execute(ddl);
-	        ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
-	        conn.createStatement().execute(ddl);
-	
-	        query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)"; 
-	        rs = conn.createStatement().executeQuery(query);
-	        assertTrue(rs.next());
-        } 
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
+            String ddl = "CREATE TABLE " + fullTableName +"  (PK1 CHAR(2) NOT NULL PRIMARY KEY, CF1.COL1 BIGINT) " + tableDDLOptions;
+            conn.createStatement().execute(ddl);
+            ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + "(COL1)";
+            conn.createStatement().execute(ddl);
+
+            query = "SELECT COUNT(COL1) FROM " + fullTableName +" WHERE COL1 IN (1,25,50,75,100)";
+            rs = conn.createStatement().executeQuery(query);
+            assertTrue(rs.next());
+        }
     }
 
     @Test
     public void testIndexWithDecimalCol() throws Exception {
-    	Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        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);
-	        String query;
-	        ResultSet rs;
+            conn.setAutoCommit(false);
+            String query;
+            ResultSet rs;
             Date date = new Date(System.currentTimeMillis());
-            
+
             createMultiCFTestTable(fullTableName, tableDDLOptions);
             populateMultiCFTestTable(fullTableName, date);
             String ddl = null;
             ddl = "CREATE " + (localIndex ? "LOCAL " : "") + "INDEX " + indexName + " ON " + fullTableName + " (decimal_pk) INCLUDE (decimal_col1, decimal_col2)";
             PreparedStatement stmt = conn.prepareStatement(ddl);
             stmt.execute();
-            
+
             query = "SELECT decimal_pk, decimal_col1, decimal_col2 from " + fullTableName ;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if(localIndex) {
@@ -970,7 +1007,7 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY FULL SCAN OVER " + fullIndexName, QueryUtil.getExplainPlan(rs));
             }
-            
+
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals(new BigDecimal("1.1"), rs.getBigDecimal(1));
@@ -985,7 +1022,54 @@ public class IndexIT extends BaseHBaseManagedTimeIT {
             assertEquals(new BigDecimal("4.3"), rs.getBigDecimal(2));
             assertEquals(new BigDecimal("5.3"), rs.getBigDecimal(3));
             assertFalse(rs.next());
-        } 
+        }
     }
-    
-}
\ No newline at end of file
+
+    /**
+     * Ensure that HTD contains table priorities correctly.
+     */
+    @Test
+    public void testTableDescriptorPriority() throws SQLException, IOException {
+        String tableName = "TBL_" + generateRandomString();
+        String indexName = "IND_" + generateRandomString();
+        String fullTableName = SchemaUtil.getTableName(TestUtil.DEFAULT_SCHEMA_NAME, tableName);
+        // Check system tables priorities.
+        try (HBaseAdmin admin = driver.getConnectionQueryServices(null, null).getAdmin()) {
+            for (HTableDescriptor htd : admin.listTables()) {
+                if (htd.getTableName().getNameAsString().startsWith(QueryConstants.SYSTEM_SCHEMA_NAME)) {
+                    String val = htd.getValue("PRIORITY");
+                    assertNotNull("PRIORITY is not set for table:" + htd, val);
+                    assertTrue(Integer.parseInt(val)
+                            >= PhoenixRpcSchedulerFactory.getMetadataPriority(config));
+                }
+            }
+
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            String ddl ="CREATE TABLE " + fullTableName + BaseTest.TEST_TABLE_SCHEMA + tableDDLOptions;
+            try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+                conn.setAutoCommit(false);
+                Statement stmt = conn.createStatement();
+                stmt.execute(ddl);
+                BaseTest.populateTestTable(fullTableName);
+                ddl = "CREATE " + (localIndex ? "LOCAL" : "") + " INDEX " + indexName
+                        + " ON " + fullTableName + " (long_col1, long_col2)"
+                        + " INCLUDE (decimal_col1, decimal_col2)";
+                stmt.execute(ddl);
+            }
+
+            HTableDescriptor dataTable = admin.getTableDescriptor(
+                    org.apache.hadoop.hbase.TableName.valueOf(fullTableName));
+            String val = dataTable.getValue("PRIORITY");
+            assertTrue(val == null || Integer.parseInt(val) < HConstants.HIGH_QOS);
+
+            if (!localIndex && mutable) {
+                HTableDescriptor indexTable = admin.getTableDescriptor(
+                        org.apache.hadoop.hbase.TableName.valueOf(indexName));
+                val = indexTable.getValue("PRIORITY");
+                assertNotNull("PRIORITY is not set for table:" + indexTable, val);
+                assertTrue(Integer.parseInt(val) >= PhoenixRpcSchedulerFactory.getIndexPriority(config));
+            }
+        }
+    }
+
+}


[14/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/8e33134f/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 4a3e92a..e226a2a 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
@@ -39,6 +39,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -256,7 +258,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
-    
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -268,7 +270,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported() {
                 @Override
@@ -284,11 +286,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
-    
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -346,7 +348,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -354,13 +356,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -368,7 +370,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -381,7 +383,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -395,7 +397,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -405,11 +407,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -522,10 +524,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -537,7 +539,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -558,7 +560,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -615,22 +617,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -642,7 +644,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -683,7 +685,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -694,7 +696,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -717,7 +719,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -725,14 +727,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -774,9 +776,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -793,7 +817,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -833,7 +857,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -855,7 +879,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -864,7 +888,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -982,7 +1006,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1006,7 +1030,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1048,12 +1072,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1103,17 +1127,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
         try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1186,18 +1210,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1221,12 +1245,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1266,10 +1291,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1327,11 +1348,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1359,10 +1380,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1401,30 +1422,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1435,26 +1456,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1468,26 +1489,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1511,7 +1532,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1541,28 +1562,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1612,7 +1633,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1624,7 +1645,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1638,8 +1659,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1663,13 +1684,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1690,7 +1715,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1700,13 +1725,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1715,9 +1740,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1725,37 +1750,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1861,7 +1886,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1890,7 +1915,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1919,7 +1944,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1943,7 +1968,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1955,7 +1980,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1977,13 +2002,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2014,8 +2039,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2033,7 +2058,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2047,7 +2072,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2071,7 +2096,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2088,7 +2113,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2131,7 +2156,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2146,7 +2171,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2158,12 +2183,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2176,7 +2201,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2186,25 +2211,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2221,7 +2246,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2254,15 +2279,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2292,34 +2317,34 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         try {
                             openConnection();
                             String noUpgradeProp = props.getProperty(PhoenixRuntime.NO_UPGRADE_ATTRIB);
-                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp)); 
+                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
-                            }   
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
+                            }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
@@ -2359,10 +2384,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2385,7 +2410,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2421,17 +2446,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
+                                        metaConnection = dropStatsTable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
                                                 PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                                 PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
@@ -2468,7 +2493,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2485,57 +2510,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                               metaConnection,
+                                               SYSTEM_STATS_NAME,
+                                               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                               PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                       + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2658,7 +2683,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + (snapshotRestored ? " after successfully restoring using snapshot"
                                                     + snapshotName
                                                     : " after restoring using snapshot "
-                                                            + snapshotName + " failed. "));
+                                                    + snapshotName + " failed. "));
                                 } finally {
                                     try {
                                         admin.close();
@@ -2679,7 +2704,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 private void ensureSystemTablesUpgraded(ReadOnlyProps props)
                         throws SQLException, IOException, IllegalArgumentException, InterruptedException {
                     if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }
@@ -2721,7 +2746,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 /**
                  * Acquire distributed mutex of sorts to make sure only one JVM is able to run the upgrade code by
                  * making use of HBase's checkAndPut api.
@@ -2732,13 +2757,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                  * wins the race will end up setting the version cell to the {@value MetaDataProtocol#MIN_SYSTEM_TABLE_TIMESTAMP}
                  * for the release.
                  * </p>
-                 * 
+                 *
                  * @return true if client won the race, false otherwise
                  * @throws IOException
                  * @throws SQLException
                  */
                 private boolean acquireUpgradeMutex(long currentServerSideTableTimestamp, byte[] sysCatalogTableName) throws IOException,
-                        SQLException {
+                SQLException {
                     Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
                     try (HTableInterface sysCatalogTable = getTable(sysCatalogTableName)) {
                         byte[] row = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
@@ -2764,7 +2789,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw Throwables.propagate(e);
         }
     }
-    
+
     private static class UpgradeInProgressException extends SQLException {
         public UpgradeInProgressException(String upgradeFrom, String upgradeTo) {
             super("Cluster is being concurrently upgraded from " + upgradeFrom + " to " + upgradeTo
@@ -2780,7 +2805,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2803,18 +2828,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2854,8 +2879,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2878,50 +2903,50 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
 
     private void scheduleRenewLeaseTasks() {
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }
@@ -2931,7 +2956,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2961,22 +2986,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                bui

<TRUNCATED>

[05/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/09ce1f51/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 4a3e92a..e226a2a 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
@@ -39,6 +39,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -85,6 +86,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -256,7 +258,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
-    
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -268,7 +270,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported() {
                 @Override
@@ -284,11 +286,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
-    
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -346,7 +348,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -354,13 +356,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -368,7 +370,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -381,7 +383,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -395,7 +397,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -405,11 +407,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -522,10 +524,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -537,7 +539,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -558,7 +560,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -615,22 +617,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -642,7 +644,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -683,7 +685,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -694,7 +696,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -717,7 +719,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -725,14 +727,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -774,9 +776,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -793,7 +817,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -833,7 +857,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -855,7 +879,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -864,7 +888,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -982,7 +1006,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1006,7 +1030,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1048,12 +1072,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1103,17 +1127,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
         try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1186,18 +1210,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1221,12 +1245,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1266,10 +1291,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1327,11 +1348,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1359,10 +1380,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1401,30 +1422,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1435,26 +1456,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1468,26 +1489,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1511,7 +1532,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1541,28 +1562,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1612,7 +1633,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1624,7 +1645,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1638,8 +1659,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1663,13 +1684,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1690,7 +1715,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1700,13 +1725,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1715,9 +1740,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1725,37 +1750,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1861,7 +1886,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1890,7 +1915,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1919,7 +1944,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1943,7 +1968,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1955,7 +1980,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1977,13 +2002,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2014,8 +2039,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2033,7 +2058,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2047,7 +2072,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2071,7 +2096,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2088,7 +2113,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2131,7 +2156,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2146,7 +2171,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2158,12 +2183,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2176,7 +2201,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2186,25 +2211,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2221,7 +2246,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2254,15 +2279,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2292,34 +2317,34 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         try {
                             openConnection();
                             String noUpgradeProp = props.getProperty(PhoenixRuntime.NO_UPGRADE_ATTRIB);
-                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp)); 
+                            boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
-                            }   
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
+                            }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
@@ -2359,10 +2384,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2385,7 +2410,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2421,17 +2446,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
+                                        metaConnection = dropStatsTable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
                                                 PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                                 PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
@@ -2468,7 +2493,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2485,57 +2510,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                               metaConnection,
+                                               SYSTEM_STATS_NAME,
+                                               MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                               PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                       + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2658,7 +2683,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                             + (snapshotRestored ? " after successfully restoring using snapshot"
                                                     + snapshotName
                                                     : " after restoring using snapshot "
-                                                            + snapshotName + " failed. "));
+                                                    + snapshotName + " failed. "));
                                 } finally {
                                     try {
                                         admin.close();
@@ -2679,7 +2704,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 private void ensureSystemTablesUpgraded(ReadOnlyProps props)
                         throws SQLException, IOException, IllegalArgumentException, InterruptedException {
                     if (!SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM, props)) { return; }
@@ -2721,7 +2746,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         }
                     }
                 }
-                
+
                 /**
                  * Acquire distributed mutex of sorts to make sure only one JVM is able to run the upgrade code by
                  * making use of HBase's checkAndPut api.
@@ -2732,13 +2757,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                  * wins the race will end up setting the version cell to the {@value MetaDataProtocol#MIN_SYSTEM_TABLE_TIMESTAMP}
                  * for the release.
                  * </p>
-                 * 
+                 *
                  * @return true if client won the race, false otherwise
                  * @throws IOException
                  * @throws SQLException
                  */
                 private boolean acquireUpgradeMutex(long currentServerSideTableTimestamp, byte[] sysCatalogTableName) throws IOException,
-                        SQLException {
+                SQLException {
                     Preconditions.checkArgument(currentServerSideTableTimestamp < MIN_SYSTEM_TABLE_TIMESTAMP);
                     try (HTableInterface sysCatalogTable = getTable(sysCatalogTableName)) {
                         byte[] row = SchemaUtil.getTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA,
@@ -2764,7 +2789,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw Throwables.propagate(e);
         }
     }
-    
+
     private static class UpgradeInProgressException extends SQLException {
         public UpgradeInProgressException(String upgradeFrom, String upgradeTo) {
             super("Cluster is being concurrently upgraded from " + upgradeFrom + " to " + upgradeTo
@@ -2780,7 +2805,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2803,18 +2828,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2854,8 +2879,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2878,50 +2903,50 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
 
     private void scheduleRenewLeaseTasks() {
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }
@@ -2931,7 +2956,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2961,22 +2986,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                bui

<TRUNCATED>

[20/21] phoenix git commit: PHOENIX-3072 Deadlock on region opening with secondary index recovery (Enis Soztutar)

Posted by el...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/714c7d12/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 352aa50..0dd341b 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
@@ -40,6 +40,7 @@ import java.lang.ref.WeakReference;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -86,6 +87,7 @@ import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.ipc.BlockingRpcCallback;
+import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
@@ -257,6 +259,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     // setting this member variable guarded by "connectionCountLock"
     private volatile ConcurrentMap<SequenceKey,Sequence> sequenceMap = Maps.newConcurrentMap();
     private KeyValueBuilder kvBuilder;
+
     private final int renewLeaseTaskFrequency;
     private final int renewLeasePoolSize;
     private final int renewLeaseThreshold;
@@ -268,7 +271,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private static interface FeatureSupported {
         boolean isSupported(ConnectionQueryServices services);
     }
-    
+
     private final Map<Feature, FeatureSupported> featureMap = ImmutableMap.<Feature, FeatureSupported>of(
             Feature.LOCAL_INDEX, new FeatureSupported() {
                 @Override
@@ -284,11 +287,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     return hbaseVersion >= PhoenixDatabaseMetaData.MIN_RENEW_LEASE_VERSION;
                 }
             });
-    
+
     private PMetaData newEmptyMetaData() {
         return new PSynchronizedMetaData(new PMetaDataImpl(INITIAL_META_DATA_TABLE_CAPACITY, getProps()));
     }
-    
+
     /**
      * Construct a ConnectionQueryServicesImpl that represents a connection to an HBase
      * cluster.
@@ -346,7 +349,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public TransactionSystemClient getTransactionSystemClient() {
         return txServiceClient;
     }
-    
+
     private void initTxServiceClient() {
         String zkQuorumServersString = this.getProps().get(TxConstants.Service.CFG_DATA_TX_ZOOKEEPER_QUORUM);
         if (zkQuorumServersString==null) {
@@ -354,13 +357,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         int timeOut = props.getInt(HConstants.ZK_SESSION_TIMEOUT, HConstants.DEFAULT_ZK_SESSION_TIMEOUT);
-        // Create instance of the tephra zookeeper client 
+        // Create instance of the tephra zookeeper client
         ZKClientService tephraZKClientService = new TephraZKClientService(zkQuorumServersString, timeOut, null, ArrayListMultimap.<String, byte[]>create());
-        
+
         ZKClientService zkClientService = ZKClientServices.delegate(
-                  ZKClients.reWatchOnExpire(
-                         ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
-                  )
+                ZKClients.reWatchOnExpire(
+                        ZKClients.retryOnFailure(tephraZKClientService, RetryStrategies.exponentialDelay(500, 2000, TimeUnit.MILLISECONDS))
+                        )
                 );
         zkClientService.startAndWait();
         ZKDiscoveryService zkDiscoveryService = new ZKDiscoveryService(zkClientService);
@@ -368,7 +371,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 config, zkDiscoveryService);
         this.txServiceClient = new TransactionServiceClient(config,pooledClientProvider);
     }
-    
+
     private void openConnection() throws SQLException {
         try {
             boolean transactionsEnabled = props.getBoolean(
@@ -381,7 +384,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             this.connection = HBaseFactoryProvider.getHConnectionFactory().createConnection(this.config);
         } catch (IOException e) {
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION)
-                .setRootCause(e).build().buildException();
+            .setRootCause(e).build().buildException();
         }
         if (this.connection.isClosed()) { // TODO: why the heck doesn't this throw above?
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ESTABLISH_CONNECTION).build().buildException();
@@ -395,7 +398,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
             throw new TableNotFoundException(SchemaUtil.getSchemaNameFromFullName(tableName), SchemaUtil.getTableNameFromFullName(tableName));
         } catch (IOException e) {
-        	throw new SQLException(e);
+            throw new SQLException(e);
         }
     }
 
@@ -405,11 +408,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             return htable.getTableDescriptor();
         } catch (IOException e) {
-            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException ||
-                e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
-              byte[][] schemaAndTableName = new byte[2][];
-              SchemaUtil.getVarChars(tableName, schemaAndTableName);
-              throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
+            if(e instanceof org.apache.hadoop.hbase.TableNotFoundException
+                    || e.getCause() instanceof org.apache.hadoop.hbase.TableNotFoundException) {
+                byte[][] schemaAndTableName = new byte[2][];
+                SchemaUtil.getVarChars(tableName, schemaAndTableName);
+                throw new TableNotFoundException(Bytes.toString(schemaAndTableName[0]), Bytes.toString(schemaAndTableName[1]));
             }
             throw new RuntimeException(e);
         } finally {
@@ -522,10 +525,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 List<HRegionLocation> locations = Lists.newArrayList();
                 byte[] currentKey = HConstants.EMPTY_START_ROW;
                 do {
-                  HRegionLocation regionLocation = connection.getRegionLocation(
-                      TableName.valueOf(tableName), currentKey, reload);
-                  locations.add(regionLocation);
-                  currentKey = regionLocation.getRegionInfo().getEndKey();
+                    HRegionLocation regionLocation = connection.getRegionLocation(
+                            TableName.valueOf(tableName), currentKey, reload);
+                    locations.add(regionLocation);
+                    currentKey = regionLocation.getRegionInfo().getEndKey();
                 } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW));
                 return locations;
             } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
@@ -537,7 +540,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     continue;
                 }
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_REGIONS_FAIL)
-                    .setRootCause(e).build().buildException();
+                .setRootCause(e).build().buildException();
             }
         }
     }
@@ -558,7 +561,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-    
+
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -615,22 +618,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     // restore the interrupt status
                     Thread.currentThread().interrupt();
                     throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION)
-                        .setRootCause(e).build().buildException(); // FIXME
+                    .setRootCause(e).build().buildException(); // FIXME
                 }
             }
             latestMetaData = metaData;
             latestMetaDataLock.notifyAll();
             return metaData;
         }
-     }
+    }
 
-	@Override
+    @Override
     public void addColumn(final PName tenantId, final String tableName, final List<PColumn> columns,
             final long tableTimeStamp, final long tableSeqNum, final boolean isImmutableRows,
             final boolean isWalDisabled, final boolean isMultitenant, final boolean storeNulls,
             final boolean isTransactional, final long updateCacheFrequency, final boolean isNamespaceMapped,
             final long resolvedTime) throws SQLException {
-	    metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
+        metaDataMutated(tenantId, tableName, tableSeqNum, new Mutator() {
             @Override
             public void mutate(PMetaData metaData) throws SQLException {
                 try {
@@ -642,7 +645,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
         });
-     }
+    }
 
     @Override
     public void removeTable(PName tenantId, final String tableName, String parentTableName, long tableTimeStamp) throws SQLException {
@@ -684,7 +687,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (tableType != PTableType.VIEW) {
             if(props.get(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB) != null){
                 columnDesc.setKeepDeletedCells(props.getBoolean(
-                    QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
+                        QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, QueryServicesOptions.DEFAULT_KEEP_DELETED_CELLS));
             }
             columnDesc.setDataBlockEncoding(SchemaUtil.DEFAULT_DATA_BLOCK_ENCODING);
             for (Entry<String,Object> entry : family.getSecond().entrySet()) {
@@ -695,7 +698,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return columnDesc;
     }
-    
+
     // Workaround HBASE-14737
     private static void setHColumnDescriptorValue(HColumnDescriptor columnDesc, String key, Object value) {
         if (HConstants.VERSIONS.equals(key)) {
@@ -718,7 +721,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return Integer.parseInt(stringValue);
     }
-    
+
     private void modifyColumnFamilyDescriptor(HColumnDescriptor hcd, Map<String,Object> props) throws SQLException {
         for (Entry<String, Object> entry : props.entrySet()) {
             String propName = entry.getKey();
@@ -726,14 +729,14 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             setHColumnDescriptorValue(hcd, propName, value);
         }
     }
-    
+
     private HTableDescriptor generateTableDescriptor(byte[] tableName, HTableDescriptor existingDesc,
             PTableType tableType, Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families,
             byte[][] splits, boolean isNamespaceMapped) throws SQLException {
         String defaultFamilyName = (String)tableProps.remove(PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME);
         HTableDescriptor tableDescriptor = (existingDesc != null) ? new HTableDescriptor(existingDesc)
-                : new HTableDescriptor(
-                        SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
+        : new HTableDescriptor(
+                SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes());
         for (Entry<String,Object> entry : tableProps.entrySet()) {
             String key = entry.getKey();
             if (!TableProperty.isPhoenixTableProperty(key)) {
@@ -775,9 +778,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
         addCoprocessors(tableName, tableDescriptor, tableType, tableProps);
+
+        // PHOENIX-3072: Set index priority if this is a system table or index table
+        if (tableType == PTableType.SYSTEM) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getMetadataPriority(config)));
+        } else if (tableType == PTableType.INDEX // Global, mutable index
+                && !isLocalIndexTable(tableDescriptor.getFamiliesKeys())
+                && !Boolean.TRUE.equals(tableProps.get(PhoenixDatabaseMetaData.IMMUTABLE_ROWS))) {
+            tableDescriptor.setValue(QueryConstants.PRIORITY,
+                    String.valueOf(PhoenixRpcSchedulerFactory.getIndexPriority(config)));
+        }
         return tableDescriptor;
     }
 
+    private boolean isLocalIndexTable(Collection<byte[]> families) {
+        // no easier way to know local index table?
+        for (byte[] family: families) {
+            if (Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    
     private void addCoprocessors(byte[] tableName, HTableDescriptor descriptor, PTableType tableType, Map<String,Object> tableProps) throws SQLException {
         // The phoenix jar must be available on HBase classpath
         int priority = props.getInt(QueryServices.COPROCESSOR_PRIORITY_ATTRIB, QueryServicesOptions.DEFAULT_COPROCESSOR_PRIORITY);
@@ -794,7 +819,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (!descriptor.hasCoprocessor(ServerCachingEndpointImpl.class.getName())) {
                 descriptor.addCoprocessor(ServerCachingEndpointImpl.class.getName(), null, priority, null);
             }
-            boolean isTransactional = 
+            boolean isTransactional =
                     Boolean.TRUE.equals(tableProps.get(TableProperty.TRANSACTIONAL.name())) ||
                     Boolean.TRUE.equals(tableProps.get(TxConstants.READ_NON_TX_DATA)); // For ALTER TABLE
             // TODO: better encapsulation for this
@@ -834,7 +859,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if(Bytes.toString(family).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                     if (!descriptor.hasCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName())) {
                         descriptor.addCoprocessor(IndexHalfStoreFileReaderGenerator.class.getName(),
-                            null, priority, null);
+                                null, priority, null);
                         break;
                     }
                 }
@@ -856,7 +881,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     descriptor.addCoprocessor(SequenceRegionObserver.class.getName(), null, priority, null);
                 }
             }
-            
+
             if (isTransactional) {
                 if (!descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.addCoprocessor(PhoenixTransactionalProcessor.class.getName(), null, priority - 10, null);
@@ -865,7 +890,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If exception on alter table to transition back to non transactional
                 if (descriptor.hasCoprocessor(PhoenixTransactionalProcessor.class.getName())) {
                     descriptor.removeCoprocessor(PhoenixTransactionalProcessor.class.getName());
-                }                
+                }
             }
         } catch (IOException e) {
             throw ServerUtil.parseServerException(e);
@@ -983,7 +1008,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         SQLException sqlE = null;
         HTableDescriptor existingDesc = null;
         boolean isMetaTable = SchemaUtil.isMetaTable(tableName);
-        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(); 
+        byte[] physicalTable = SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes();
         boolean tableExist = true;
         try (HBaseAdmin admin = getAdmin()) {
             final String quorum = ZKConfig.getZKQuorumServersString(config);
@@ -1007,7 +1032,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
             if (!tableExist) {
                 if (newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES) != null && Boolean.TRUE.equals(
-                    PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
+                        PBoolean.INSTANCE.toObject(newDesc.getValue(MetaDataUtil.IS_LOCAL_INDEX_TABLE_PROP_BYTES)))) {
                     newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
                 }
                 // Remove the splitPolicy attribute to prevent HBASE-12570
@@ -1049,12 +1074,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 } else {
                     for(Pair<byte[],Map<String,Object>> family: families) {
                         if ((newDesc.getValue(HTableDescriptor.SPLIT_POLICY)==null || !newDesc.getValue(HTableDescriptor.SPLIT_POLICY).equals(
-                            IndexRegionSplitPolicy.class.getName()))
+                                IndexRegionSplitPolicy.class.getName()))
                                 && Bytes.toString(family.getFirst()).startsWith(
-                                    QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                                   newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
-                                   break;
-                           }
+                                        QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                            newDesc.setValue(HTableDescriptor.SPLIT_POLICY, IndexRegionSplitPolicy.class.getName());
+                            break;
+                        }
                     }
                 }
 
@@ -1103,18 +1128,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private void modifyTable(byte[] tableName, HTableDescriptor newDesc, boolean shouldPoll) throws IOException,
     InterruptedException, TimeoutException, SQLException {
-    	try (HBaseAdmin admin = getAdmin()) {
-    		if (!allowOnlineTableSchemaUpdate()) {
-    			admin.disableTable(tableName);
-    			admin.modifyTable(tableName, newDesc);
-    			admin.enableTable(tableName);
-    		} else {
-    			admin.modifyTable(tableName, newDesc);
-    			if (shouldPoll) {
-    			    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
-    			}
-    		}
-    	}
+        try (HBaseAdmin admin = getAdmin()) {
+            if (!allowOnlineTableSchemaUpdate()) {
+                admin.disableTable(tableName);
+                admin.modifyTable(tableName, newDesc);
+                admin.enableTable(tableName);
+            } else {
+                admin.modifyTable(tableName, newDesc);
+                if (shouldPoll) {
+                    pollForUpdatedTableDescriptor(admin, newDesc, tableName);
+                }
+            }
+        }
     }
 
     private static boolean hasIndexWALCodec(Long serverVersion) {
@@ -1187,18 +1212,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (isTableNamespaceMappingEnabled != SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                     getProps())) { throw new SQLExceptionInfo.Builder(
                             SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
-                                    .setMessage(
-                                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
-                                                    + " is consitent on client and server.")
-                                    .build().buildException(); }
+                    .setMessage(
+                            "Ensure that config " + QueryServices.IS_NAMESPACE_MAPPING_ENABLED
+                            + " is consitent on client and server.")
+                            .build().buildException(); }
             lowestClusterHBaseVersion = minHBaseVersion;
         } catch (SQLException e) {
             throw e;
         } catch (Throwable t) {
             // This is the case if the "phoenix.jar" is not on the classpath of HBase on the region server
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INCOMPATIBLE_CLIENT_SERVER_JAR).setRootCause(t)
-                .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
-                .build().buildException();
+            .setMessage("Ensure that " + QueryConstants.DEFAULT_COPROCESS_PATH + " is put on the classpath of HBase in every region server: " + t.getMessage())
+            .build().buildException();
         } finally {
             if (ht != null) {
                 try {
@@ -1222,12 +1247,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             Batch.Call<MetaDataService, MetaDataResponse> callable) throws SQLException {
         return metaDataCoprocessorExec(tableKey, callable, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES);
     }
-        /**
-         * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
-         * (due to a table split)
-         */
-        private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
-                Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
+
+    /**
+     * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
+     * (due to a table split)
+     */
+    private MetaDataMutationResult metaDataCoprocessorExec(byte[] tableKey,
+            Batch.Call<MetaDataService, MetaDataResponse> callable, byte[] tableName) throws SQLException {
 
         try {
             boolean retried = false;
@@ -1267,10 +1293,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     private void ensureViewIndexTableCreated(byte[] physicalTableName, Map<String, Object> tableProps,
             List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, long timestamp,
             boolean isNamespaceMapped) throws SQLException {
-        Long maxFileSize = (Long)tableProps.get(HTableDescriptor.MAX_FILESIZE);
-        if (maxFileSize == null) {
-            maxFileSize = this.props.getLong(HConstants.HREGION_MAX_FILESIZE, HConstants.DEFAULT_MAX_FILE_SIZE);
-        }
         byte[] physicalIndexName = MetaDataUtil.getViewIndexPhysicalName(physicalTableName);
 
         tableProps.put(MetaDataUtil.IS_VIEW_INDEX_TABLE_PROP_NAME, TRUE_BYTES_AS_STRING);
@@ -1328,11 +1350,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     for(HColumnDescriptor cf : desc.getColumnFamilies()) {
                         if(cf.getNameAsString().startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
                             columnFamiles.add(cf.getNameAsString());
-                        }  
+                        }
                     }
                     for(String cf: columnFamiles) {
                         admin.deleteColumn(physicalTableName, cf);
-                    }  
+                    }
                     clearTableRegionCache(physicalTableName);
                     wasDeleted = true;
                 }
@@ -1360,10 +1382,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableName = physicalTableName != null ? physicalTableName : SchemaUtil.getTableNameAsBytes(schemaBytes, tableBytes);
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
-               if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
-                       localIndexTable = true;
-                       break;
-               }
+            if(Bytes.toString(family.getFirst()).startsWith(QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX)) {
+                localIndexTable = true;
+                break;
+            }
         }
         if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
@@ -1402,30 +1424,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             ensureViewIndexTableCreated(
                     SchemaUtil.getPhysicalHBaseTableName(tableName, isNamespaceMapped, tableType).getBytes(),
                     tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
-                    MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
+                            MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        CreateTableRequest build = builder.build();
-						instance.createTable(controller, build, rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                CreateTableRequest build = builder.build();
+                instance.createTable(controller, build, rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
         });
         return result;
     }
@@ -1436,26 +1458,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    GetTableRequest.Builder builder = GetTableRequest.newBuilder();
-                    builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
-                    builder.setSchemaName(ByteStringer.wrap(schemaBytes));
-                    builder.setTableName(ByteStringer.wrap(tableBytes));
-                    builder.setTableTimestamp(tableTimestamp);
-                    builder.setClientTimestamp(clientTimestamp);
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.getTable(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                GetTableRequest.Builder builder = GetTableRequest.newBuilder();
+                builder.setTenantId(ByteStringer.wrap(tenantIdBytes));
+                builder.setSchemaName(ByteStringer.wrap(schemaBytes));
+                builder.setTableName(ByteStringer.wrap(tableBytes));
+                builder.setTableTimestamp(tableTimestamp);
+                builder.setClientTimestamp(clientTimestamp);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.getTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
                 }
-            });
+                return rpcCallback.get();
+            }
+        });
     }
 
     @Override
@@ -1469,26 +1491,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantIdBytes, schemaBytes, tableBytes);
         final MetaDataMutationResult result =  metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropTableRequest.Builder builder = DropTableRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setTableType(tableType.getSerializedValue());
-                        builder.setCascade(cascade);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropTable(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                });
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropTableRequest.Builder builder = DropTableRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setTableType(tableType.getSerializedValue());
+                builder.setCascade(cascade);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropTable(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
 
         final MutationCode code = result.getMutationCode();
         switch(code) {
@@ -1512,7 +1534,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         default:
             break;
         }
-          return result;
+        return result;
     }
 
     /*
@@ -1542,28 +1564,28 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] functionBytes = rowKeyMetadata[PhoenixDatabaseMetaData.FUNTION_NAME_INDEX];
         byte[] functionKey = SchemaUtil.getFunctionKey(tenantIdBytes, functionBytes);
-        
+
         final MetaDataMutationResult result =  metaDataCoprocessorExec(functionKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
-                        for (Mutation m : functionData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setIfExists(ifExists);
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.dropFunction(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
-                    }
-                }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropFunctionRequest.Builder builder = DropFunctionRequest.newBuilder();
+                for (Mutation m : functionData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
+                }
+                builder.setIfExists(ifExists);
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropFunction(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        }, PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES);
         return result;
     }
 
@@ -1608,7 +1630,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         String name = Bytes
                 .toString(SchemaUtil.getParentTableNameFromIndexTable(physicalIndexTableName,
                         MetaDataUtil.VIEW_INDEX_TABLE_PREFIX))
-                .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+                        .replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
         PTable table = getTable(tenantId, name, timestamp);
         ensureViewIndexTableCreated(table, timestamp, isNamespaceMapped);
     }
@@ -1620,7 +1642,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throwConnectionClosedIfNullMetaData();
             table = metadata.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
             if (table.getTimeStamp() >= timestamp) { // Table in cache is newer than client timestamp which shouldn't be
-                                                     // the case
+                // the case
                 throw new TableNotFoundException(table.getSchemaName().getString(), table.getTableName().getString());
             }
         } catch (TableNotFoundException e) {
@@ -1634,8 +1656,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return table;
     }
 
-	private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
-			throws SQLException {
+    private void ensureViewIndexTableCreated(PTable table, long timestamp, boolean isNamespaceMapped)
+            throws SQLException {
         byte[] physicalTableName = table.getPhysicalName().getBytes();
         HTableDescriptor htableDesc = this.getTableDescriptor(physicalTableName);
         Map<String,Object> tableProps = createPropertiesMap(htableDesc.getValues());
@@ -1659,13 +1681,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             splits = SaltingUtil.getSalteByteSplitPoints(table.getBucketNum());
         }
 
+        // Transfer over table values into tableProps
+        // TODO: encapsulate better
+        tableProps.put(PhoenixDatabaseMetaData.TRANSACTIONAL, table.isTransactional());
+        tableProps.put(PhoenixDatabaseMetaData.IMMUTABLE_ROWS, table.isImmutableRows());
         ensureViewIndexTableCreated(physicalTableName, tableProps, families, splits, timestamp, isNamespaceMapped);
     }
-    
+
     @Override
     public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded) throws SQLException {
-    	List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
-    	Map<String, Object> tableProps = new HashMap<String, Object>();
+        List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
+        Map<String, Object> tableProps = new HashMap<String, Object>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
         Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
         boolean nonTxToTx = false;
@@ -1686,7 +1712,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 updateDescriptorForTx(table, tableProps, tableDescriptor, Boolean.TRUE.toString(), tableDescriptors, origTableDescriptors);
             }
         }
-        
+
         boolean success = false;
         boolean metaDataUpdated = !tableDescriptors.isEmpty();
         boolean pollingNeeded = !(!tableProps.isEmpty() && families.isEmpty() && colFamiliesForPColumnsToBeAdded.isEmpty());
@@ -1696,13 +1722,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if (table.getType() == PTableType.VIEW) {
                 boolean canViewsAddNewCF = props.getBoolean(QueryServices.ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE,
                         QueryServicesOptions.DEFAULT_ALLOW_VIEWS_ADD_NEW_CF_BASE_TABLE);
-                // When adding a column to a view, base physical table should only be modified when new column families are being added.  
+                // When adding a column to a view, base physical table should only be modified when new column families are being added.
                 modifyHTable = canViewsAddNewCF && !existingColumnFamiliesForBaseTable(table.getPhysicalName()).containsAll(colFamiliesForPColumnsToBeAdded);
             }
             if (modifyHTable) {
                 sendHBaseMetaData(tableDescriptors, pollingNeeded);
             }
-            
+
             // Special case for call during drop table to ensure that the empty column family exists.
             // In this, case we only include the table header row, as until we add schemaBytes and tableBytes
             // as args to this function, we have no way of getting them in this case.
@@ -1711,9 +1737,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             if ((tableMetaData.isEmpty()) || (tableMetaData.size() == 1 && tableMetaData.get(0).isEmpty())) {
                 return new MetaDataMutationResult(MutationCode.NO_OP, System.currentTimeMillis(), table);
             }
-             byte[][] rowKeyMetaData = new byte[3][];
-             PTableType tableType = table.getType();
-    
+            byte[][] rowKeyMetaData = new byte[3][];
+            PTableType tableType = table.getType();
+
             Mutation m = tableMetaData.get(0);
             byte[] rowKey = m.getRow();
             SchemaUtil.getVarChars(rowKey, rowKeyMetaData);
@@ -1721,37 +1747,37 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaBytes = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             byte[] tableBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-            
+
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             result =  metaDataCoprocessorExec(tableKey,
-                new Batch.Call<MetaDataService, MetaDataResponse>() {
-                    @Override
-                    public MetaDataResponse call(MetaDataService instance) throws IOException {
-                        ServerRpcController controller = new ServerRpcController();
-                        BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                                new BlockingRpcCallback<MetaDataResponse>();
-                        AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
-                        for (Mutation m : tableMetaData) {
-                            MutationProto mp = ProtobufUtil.toProto(m);
-                            builder.addTableMetadataMutations(mp.toByteString());
-                        }
-                        builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                        instance.addColumn(controller, builder.build(), rpcCallback);
-                        if(controller.getFailedOn() != null) {
-                            throw controller.getFailedOn();
-                        }
-                        return rpcCallback.get();
+                    new Batch.Call<MetaDataService, MetaDataResponse>() {
+                @Override
+                public MetaDataResponse call(MetaDataService instance) throws IOException {
+                    ServerRpcController controller = new ServerRpcController();
+                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                            new BlockingRpcCallback<MetaDataResponse>();
+                    AddColumnRequest.Builder builder = AddColumnRequest.newBuilder();
+                    for (Mutation m : tableMetaData) {
+                        MutationProto mp = ProtobufUtil.toProto(m);
+                        builder.addTableMetadataMutations(mp.toByteString());
                     }
-                });
-    
+                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    instance.addColumn(controller, builder.build(), rpcCallback);
+                    if(controller.getFailedOn() != null) {
+                        throw controller.getFailedOn();
+                    }
+                    return rpcCallback.get();
+                }
+            });
+
             if (result.getMutationCode() == MutationCode.COLUMN_NOT_FOUND || result.getMutationCode() == MutationCode.TABLE_ALREADY_EXISTS) { // Success
                 success = true;
                 // Flush the table if transitioning DISABLE_WAL from TRUE to FALSE
                 if (  MetaDataUtil.getMutationValue(m,PhoenixDatabaseMetaData.DISABLE_WAL_BYTES, kvBuilder, ptr)
-                   && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
+                        && Boolean.FALSE.equals(PBoolean.INSTANCE.toObject(ptr))) {
                     flushTable(table.getPhysicalName().getBytes());
                 }
-    
+
                 if (tableType == PTableType.TABLE) {
                     // If we're changing MULTI_TENANT to true or false, create or drop the view index table
                     if (MetaDataUtil.getMutationValue(m, PhoenixDatabaseMetaData.MULTI_TENANT_BYTES, kvBuilder, ptr)){
@@ -1857,7 +1883,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     private void sendHBaseMetaData(Set<HTableDescriptor> tableDescriptors, boolean pollingNeeded) throws SQLException {
         SQLException sqlE = null;
         for (HTableDescriptor descriptor : tableDescriptors) {
@@ -1886,7 +1912,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         this.addCoprocessors(tableDescriptor.getName(), tableDescriptor, tableType, tableProps);
     }
-    
+
     private Pair<HTableDescriptor,HTableDescriptor> separateAndValidateProperties(PTable table, Map<String, List<Pair<String, Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<Pair<byte[], Map<String, Object>>> families, Map<String, Object> tableProps) throws SQLException {
         Map<String, Map<String, Object>> stmtFamiliesPropsMap = new HashMap<>(properties.size());
         Map<String,Object> commonFamilyProps = new HashMap<>();
@@ -1915,7 +1941,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY)) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY)
                             .setMessage("Column Family: " + family + ", Property: " + propName).build()
-                            .buildException(); 
+                            .buildException();
                         }
                         tableProps.put(propName, propValue);
                     } else {
@@ -1939,7 +1965,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                 }
                             } else {
                                 // invalid property - neither of HTableProp, HColumnProp or PhoenixTableProp
-                                // FIXME: This isn't getting triggered as currently a property gets evaluated 
+                                // FIXME: This isn't getting triggered as currently a property gets evaluated
                                 // as HTableProp if its neither HColumnProp or PhoenixTableProp.
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_PROPERTY)
                                 .setMessage("Column Family: " + family + ", Property: " + propName).build()
@@ -1951,7 +1977,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 if (!colFamilyPropsMap.isEmpty()) {
                     stmtFamiliesPropsMap.put(family, colFamilyPropsMap);
                 }
-  
+
             }
         }
         commonFamilyProps = Collections.unmodifiableMap(commonFamilyProps);
@@ -1973,13 +1999,13 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         m.putAll(commonFamilyProps);
                         allFamiliesProps.put(colFamily, m);
                     } else if (isAddingPkColOnly) {
-                        // Setting HColumnProperty for a pk column is invalid 
+                        // Setting HColumnProperty for a pk column is invalid
                         // because it will be part of the row key and not a key value column family.
-                        // However, if both pk cols as well as key value columns are getting added 
+                        // However, if both pk cols as well as key value columns are getting added
                         // together, then its allowed. The above if block will make sure that we add properties
                         // only for the kv cols and not pk cols.
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.SET_UNSUPPORTED_PROP_ON_ALTER_TABLE)
-                                .build().buildException();
+                        .build().buildException();
                     }
                 }
             }
@@ -2010,8 +2036,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
 
         // case when there is a column family being added but there are no props
-        // For ex - in DROP COLUMN when a new empty CF needs to be added since all 
-        // the columns of the existing empty CF are getting dropped. Or the case 
+        // For ex - in DROP COLUMN when a new empty CF needs to be added since all
+        // the columns of the existing empty CF are getting dropped. Or the case
         // when one is just adding a column for a column family like this:
         // ALTER TABLE ADD CF.COL
         for (String cf : colFamiliesForPColumnsToBeAdded) {
@@ -2029,7 +2055,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.VIEW_WITH_PROPERTIES).build()
             .buildException();
         }
-        
+
         HTableDescriptor newTableDescriptor = null;
         HTableDescriptor origTableDescriptor = null;
         if (!allFamiliesProps.isEmpty() || !tableProps.isEmpty()) {
@@ -2043,7 +2069,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 }
             }
             if (addingColumns) {
-                // Make sure that all the CFs of the table have the same TTL as the empty CF. 
+                // Make sure that all the CFs of the table have the same TTL as the empty CF.
                 setTTLForNewCFs(allFamiliesProps, table, newTableDescriptor, newTTL);
             }
             // Set TTL on all table column families, even if they're not referenced here
@@ -2067,7 +2093,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     if (isTransactional) {
                         defaultTxMaxVersions = newTableDescriptor.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getMaxVersions();
                     } else {
-                        defaultTxMaxVersions = 
+                        defaultTxMaxVersions =
                                 this.getProps().getInt(
                                         QueryServices.MAX_VERSIONS_TRANSACTIONAL_ATTRIB,
                                         QueryServicesOptions.DEFAULT_MAX_VERSIONS_TRANSACTIONAL);
@@ -2084,7 +2110,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     }
                 }
             }
-            // Set Tephra's TTL property based on HBase property if we're 
+            // Set Tephra's TTL property based on HBase property if we're
             // transitioning to become transactional or setting TTL on
             // an already transactional table.
             if (isOrWillBeTransactional) {
@@ -2127,7 +2153,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return new Pair<>(origTableDescriptor, newTableDescriptor);
     }
-    
+
     private void checkTransactionalVersionsValue(HColumnDescriptor colDescriptor) throws SQLException {
         int maxVersions = colDescriptor.getMaxVersions();
         if (maxVersions <= 1) {
@@ -2142,7 +2168,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         PTable table = latestMetaData.getTableRef(new PTableKey(null, baseTableName.getString())).getTable();
         return existingColumnFamilies(table);
     }
-    
+
     private HashSet<String> existingColumnFamilies(PTable table) {
         List<PColumnFamily> cfs = table.getColumnFamilies();
         HashSet<String> cfNames = new HashSet<>(cfs.size());
@@ -2154,12 +2180,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     private static int getTTL(PTable table, HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         // If we're setting TTL now, then use that value. Otherwise, use empty column family value
-        int ttl = newTTL != null ? newTTL 
+        int ttl = newTTL != null ? newTTL
                 : tableDesc.getFamily(SchemaUtil.getEmptyColumnFamily(table)).getTimeToLive();
         return ttl;
     }
-    
-    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table, 
+
+    private static void setTTLForNewCFs(Map<String, Map<String, Object>> familyProps, PTable table,
             HTableDescriptor tableDesc, Integer newTTL) throws SQLException {
         if (!familyProps.isEmpty()) {
             int ttl = getTTL(table, tableDesc, newTTL);
@@ -2172,7 +2198,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
-    
+
     @Override
     public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
@@ -2182,25 +2208,25 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
-            new Batch.Call<MetaDataService, MetaDataResponse>() {
-                @Override
-                public MetaDataResponse call(MetaDataService instance) throws IOException {
-                    ServerRpcController controller = new ServerRpcController();
-                    BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                            new BlockingRpcCallback<MetaDataResponse>();
-                    DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
-                    for (Mutation m : tableMetaData) {
-                        MutationProto mp = ProtobufUtil.toProto(m);
-                        builder.addTableMetadataMutations(mp.toByteString());
-                    }
-                    builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                    instance.dropColumn(controller, builder.build(), rpcCallback);
-                    if(controller.getFailedOn() != null) {
-                        throw controller.getFailedOn();
-                    }
-                    return rpcCallback.get();
+                new Batch.Call<MetaDataService, MetaDataResponse>() {
+            @Override
+            public MetaDataResponse call(MetaDataService instance) throws IOException {
+                ServerRpcController controller = new ServerRpcController();
+                BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                        new BlockingRpcCallback<MetaDataResponse>();
+                DropColumnRequest.Builder builder = DropColumnRequest.newBuilder();
+                for (Mutation m : tableMetaData) {
+                    MutationProto mp = ProtobufUtil.toProto(m);
+                    builder.addTableMetadataMutations(mp.toByteString());
                 }
-            });
+                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                instance.dropColumn(controller, builder.build(), rpcCallback);
+                if(controller.getFailedOn() != null) {
+                    throw controller.getFailedOn();
+                }
+                return rpcCallback.get();
+            }
+        });
         final MutationCode code = result.getMutationCode();
         switch(code) {
         case TABLE_ALREADY_EXISTS:
@@ -2217,7 +2243,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         return result;
 
     }
-    
+
     /**
      * This closes the passed connection.
      */
@@ -2250,15 +2276,15 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return metaConnection;
     }
-    
-    /** 
+
+    /**
      * Keeping this to use for further upgrades. This method closes the oldMetaConnection.
      */
     private PhoenixConnection addColumnsIfNotExists(PhoenixConnection oldMetaConnection,
             String tableName, long timestamp, String columns) throws SQLException {
         return addColumn(oldMetaConnection, tableName, timestamp, columns, true);
     }
-    
+
     @Override
     public void init(final String url, final Properties props) throws SQLException {
         try {
@@ -2291,31 +2317,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             boolean upgradeSystemTables = !Boolean.TRUE.equals(Boolean.valueOf(noUpgradeProp));
                             Properties scnProps = PropertiesUtil.deepCopy(props);
                             scnProps.setProperty(
-                                PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                                Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
+                                    PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                                    Long.toString(MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP));
                             scnProps.remove(PhoenixRuntime.TENANT_ID_ATTRIB);
                             String globalUrl = JDBCUtil.removeProperty(url, PhoenixRuntime.TENANT_ID_ATTRIB);
                             metaConnection = new PhoenixConnection(
-                                ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
+                                    ConnectionQueryServicesImpl.this, globalUrl, scnProps, newEmptyMetaData());
                             try (HBaseAdmin admin = getAdmin()) {
                                 boolean mappedSystemCatalogExists = admin
                                         .tableExists(SchemaUtil.getPhysicalTableName(SYSTEM_CATALOG_NAME_BYTES, true));
                                 if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                    ConnectionQueryServicesImpl.this.getProps())) {
+                                        ConnectionQueryServicesImpl.this.getProps())) {
                                     if (admin.tableExists(SYSTEM_CATALOG_NAME_BYTES)) {
-                                        //check if the server is already updated and have namespace config properly set. 
+                                        //check if the server is already updated and have namespace config properly set.
                                         checkClientServerCompatibility(SYSTEM_CATALOG_NAME_BYTES);
                                     }
                                     ensureSystemTablesUpgraded(ConnectionQueryServicesImpl.this.getProps());
                                 } else if (mappedSystemCatalogExists) { throw new SQLExceptionInfo.Builder(
-                                    SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
+                                        SQLExceptionCode.INCONSISTENET_NAMESPACE_MAPPING_PROPERTIES)
                                 .setMessage("Cannot initiate connection as "
                                         + SchemaUtil.getPhysicalTableName(
-                                            SYSTEM_CATALOG_NAME_BYTES, true)
-                                            + " is found but client does not have "
-                                            + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
-                                            .build().buildException(); }
-                            }   
+                                                SYSTEM_CATALOG_NAME_BYTES, true)
+                                                + " is found but client does not have "
+                                                + IS_NAMESPACE_MAPPING_ENABLED + " enabled")
+                                                .build().buildException(); }
+                            }
                             try {
                                 metaConnection.createStatement().executeUpdate(QueryConstants.CREATE_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
@@ -2355,10 +2381,10 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
 
                                     // If the server side schema is before MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0 then
-                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too. 
-                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed, 
-                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all 
-                                    // the column names that have been added to SYSTEM.CATALOG since 4.0. 
+                                    // we need to add INDEX_TYPE and INDEX_DISABLE_TIMESTAMP columns too.
+                                    // TODO: Once https://issues.apache.org/jira/browse/PHOENIX-1614 is fixed,
+                                    // we should just have a ALTER TABLE ADD IF NOT EXISTS statement with all
+                                    // the column names that have been added to SYSTEM.CATALOG since 4.0.
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         columnsToAdd = addColumn(columnsToAdd, PhoenixDatabaseMetaData.INDEX_TYPE + " " + PUnsignedTinyint.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP + " " + PLong.INSTANCE.getSqlTypeName());
@@ -2381,7 +2407,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                                     MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_5_0, columnsToAdd, false);
                                             upgradeTo4_5_0(metaConnection);
                                         } catch (ColumnAlreadyExistsException ignored) {
-                                            /* 
+                                            /*
                                              * Upgrade to 4.5 is a slightly special case. We use the fact that the column
                                              * BASE_COLUMN_COUNT is already part of the meta-data schema as the signal that
                                              * the server side upgrade has finished or is in progress.
@@ -2417,17 +2443,17 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     }
                                     if(currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0) {
                                         // Drop old stats table so that new stats table is created
-                                        metaConnection = dropStatsTable(metaConnection, 
+                                        metaConnection = dropStatsTable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 4);
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 3,
                                                 PhoenixDatabaseMetaData.TRANSACTIONAL + " " + PBoolean.INSTANCE.getSqlTypeName());
-                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG, 
+                                        metaConnection = addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 2,
                                                 PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY + " " + PLong.INSTANCE.getSqlTypeName());
-                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection, 
+                                        metaConnection = setImmutableTableIndexesImmutable(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0 - 1);
-                                        metaConnection = updateSystemCatalogTimestamp(metaConnection, 
+                                        metaConnection = updateSystemCatalogTimestamp(metaConnection,
                                                 MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         ConnectionQueryServicesImpl.this.removeTable(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0);
                                         clearCache();
@@ -2464,7 +2490,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             }
 
                             int nSaltBuckets = ConnectionQueryServicesImpl.this.props.getInt(QueryServices.SEQUENCE_SALT_BUCKETS_ATTRIB,
-                                QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
+                                    QueryServicesOptions.DEFAULT_SEQUENCE_TABLE_SALT_BUCKETS);
                             try {
                                 String createSequenceTable = Sequence.getCreateTableStatement(nSaltBuckets);
                                 metaConnection.createStatement().executeUpdate(createSequenceTable);
@@ -2481,57 +2507,57 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_1_0) {
                                         // If the table time stamp is before 4.1.0 then we need to add below columns
                                         // to the SYSTEM.SEQUENCE table.
-                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName() 
+                                        String columnsToAdd = PhoenixDatabaseMetaData.MIN_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.MAX_VALUE + " " + PLong.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.CYCLE_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName()
                                                 + ", " + PhoenixDatabaseMetaData.LIMIT_REACHED_FLAG + " " + PBoolean.INSTANCE.getSqlTypeName();
                                         addColumnsIfNotExists(metaConnection, PhoenixDatabaseMetaData.SYSTEM_CATALOG,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP, columnsToAdd);
                                     }
                                     // If the table timestamp is before 4.2.1 then run the upgrade script
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_2_1) {
                                         if (UpgradeUtil.upgradeSequenceTable(metaConnection, nSaltBuckets, e.getTable())) {
                                             metaConnection.removeTable(null,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
-                                                PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
-                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_SCHEMA_BYTES,
+                                                   PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE_BYTES,
+                                                   MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP);
                                             clearTableRegionCache(PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_NAME_BYTES);
                                         }
                                         nSequenceSaltBuckets = nSaltBuckets;
-                                    } else { 
+                                    } else {
                                         nSequenceSaltBuckets = getSaltBuckets(e);
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_STATS_TABLE_METADATA);
+                                        QueryConstants.CREATE_STATS_TABLE_METADATA);
                             } catch (NewerTableAlreadyExistsException ignore) {
                             } catch(TableAlreadyExistsException e) {
                                 if (upgradeSystemTables) {
                                     long currentServerSideTableTimeStamp = e.getTable().getTimeStamp();
                                     if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_3_0) {
                                         metaConnection = addColumnsIfNotExists(
-                                            metaConnection,
-                                            SYSTEM_STATS_NAME,
-                                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
-                                            PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
-                                                    + PLong.INSTANCE.getSqlTypeName());
+                                                metaConnection,
+                                                SYSTEM_STATS_NAME,
+                                                MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP,
+                                                PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT + " "
+                                                        + PLong.INSTANCE.getSqlTypeName());
                                     }
                                 }
                             }
                             try {
                                 metaConnection.createStatement().executeUpdate(
-                                    QueryConstants.CREATE_FUNCTION_METADATA);
+                                        QueryConstants.CREATE_FUNCTION_METADATA);
                             } catch (NewerTableAlreadyExistsException e) {
                             } catch (TableAlreadyExistsException e) {
                             }
                             if (SchemaUtil.isNamespaceMappingEnabled(PTableType.SYSTEM,
-                                ConnectionQueryServicesImpl.this.getProps())) {
+                                    ConnectionQueryServicesImpl.this.getProps())) {
                                 try {
                                     metaConnection.createStatement().executeUpdate("CREATE SCHEMA IF NOT EXISTS "
                                             + PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA);
@@ -2776,7 +2802,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         return tableNames;
     }
-    
+
     private String addColumn(String columnsToAddSoFar, String columns) {
         if (columnsToAddSoFar == null || columnsToAddSoFar.isEmpty()) {
             return columns;
@@ -2789,12 +2815,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         if (isRenewingLeasesEnabled()) {
             ThreadFactory threadFactory =
                     new ThreadFactoryBuilder().setDaemon(true)
-                            .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
+                    .setNameFormat("PHOENIX-SCANNER-RENEW-LEASE" + "-thread-%s").build();
             renewLeaseExecutor =
                     Executors.newScheduledThreadPool(renewLeasePoolSize, threadFactory);
             for (LinkedBlockingQueue<WeakReference<PhoenixConnection>> q : connectionQueues) {
                 renewLeaseExecutor.scheduleAtFixedRate(new RenewLeaseTask(q), 0,
-                    renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
+                        renewLeaseTaskFrequency, TimeUnit.MILLISECONDS);
             }
         }
     }   
@@ -2813,18 +2839,18 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" + 
-                    "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" + 
-                    "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" + 
-                    " A.TENANT_ID = B.TENANT_ID AND \n" + 
-                    " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" + 
-                    " A.TABLE_NAME = B.TABLE_NAME AND\n" + 
-                    " A.COLUMN_NAME = B.COLUMN_NAME AND\n" + 
-                    " B.LINK_TYPE = 1\n" + 
-                    ")\n" + 
-                    "WHERE A.COLUMN_FAMILY IS NULL AND\n" + 
-                    " B.COLUMN_FAMILY IS NOT NULL AND\n" + 
-                    " A.IMMUTABLE_ROWS = TRUE");
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, IMMUTABLE_ROWS)\n" +
+                            "SELECT A.TENANT_ID, A.TABLE_SCHEM,B.COLUMN_FAMILY,null,null,true\n" +
+                            "FROM SYSTEM.CATALOG A JOIN SYSTEM.CATALOG B ON (\n" +
+                            " A.TENANT_ID = B.TENANT_ID AND \n" +
+                            " A.TABLE_SCHEM = B.TABLE_SCHEM AND\n" +
+                            " A.TABLE_NAME = B.TABLE_NAME AND\n" +
+                            " A.COLUMN_NAME = B.COLUMN_NAME AND\n" +
+                            " B.LINK_TYPE = 1\n" +
+                            ")\n" +
+                            "WHERE A.COLUMN_FAMILY IS NULL AND\n" +
+                            " B.COLUMN_FAMILY IS NOT NULL AND\n" +
+                            " A.IMMUTABLE_ROWS = TRUE");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2864,8 +2890,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         try {
             metaConnection.setAutoCommit(true);
             metaConnection.createStatement().execute(
-                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" + 
-                    "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)"); 
+                    "UPSERT INTO SYSTEM.CATALOG(TENANT_ID, TABLE_SCHEM, TABLE_NAME, COLUMN_NAME, COLUMN_FAMILY, DISABLE_WAL)\n" +
+                            "VALUES (NULL, '" + QueryConstants.SYSTEM_SCHEMA_NAME + "','" + PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE + "', NULL, NULL, FALSE)");
         } catch (SQLException e) {
             logger.warn("exception during upgrading stats table:" + e);
             sqlE = e;
@@ -2888,46 +2914,46 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     private PhoenixConnection dropStatsTable(PhoenixConnection oldMetaConnection, long timestamp)
-			throws SQLException, IOException {
-		Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
-		props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
-		PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
-		SQLException sqlE = null;
-		boolean wasCommit = metaConnection.getAutoCommit();
-		try {
-			metaConnection.setAutoCommit(true);
-			metaConnection.createStatement()
-					.executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
-							+ PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
-							+ "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
-							+ PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
-		} catch (SQLException e) {
-			logger.warn("exception during upgrading stats table:" + e);
-			sqlE = e;
-		} finally {
-			try {
-				metaConnection.setAutoCommit(wasCommit);
-				oldMetaConnection.close();
-			} catch (SQLException e) {
-				if (sqlE != null) {
-					sqlE.setNextException(e);
-				} else {
-					sqlE = e;
-				}
-			}
-			if (sqlE != null) {
-				throw sqlE;
-			}
-		}
-		return metaConnection;
-	}
-    
+            throws SQLException, IOException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        SQLException sqlE = null;
+        boolean wasCommit = metaConnection.getAutoCommit();
+        try {
+            metaConnection.setAutoCommit(true);
+            metaConnection.createStatement()
+            .executeUpdate("DELETE FROM " + PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " WHERE "
+                    + PhoenixDatabaseMetaData.TABLE_NAME + "='" + PhoenixDatabaseMetaData.SYSTEM_STATS_TABLE
+                    + "' AND " + PhoenixDatabaseMetaData.TABLE_SCHEM + "='"
+                    + PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME + "'");
+        } catch (SQLException e) {
+            logger.warn("exception during upgrading stats table:" + e);
+            sqlE = e;
+        } finally {
+            try {
+                metaConnection.setAutoCommit(wasCommit);
+                oldMetaConnection.close();
+            } catch (SQLException e) {
+                if (sqlE != null) {
+                    sqlE.setNextException(e);
+                } else {
+                    sqlE = e;
+                }
+            }
+            if (sqlE != null) {
+                throw sqlE;
+            }
+        }
+        return metaConnection;
+    }
+
     private static int getSaltBuckets(TableAlreadyExistsException e) {
         PTable table = e.getTable();
         Integer sequenceSaltBuckets = table == null ? null : table.getBucketNum();
         return sequenceSaltBuckets == null ? 0 : sequenceSaltBuckets;
     }
-    
+
     @Override
     public MutationState updateData(MutationPlan plan) throws SQLException {
         MutationState state = plan.execute();
@@ -2957,22 +2983,22 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     .getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, this.getProps()).getName());
             try {
                 final Map<byte[], Long> results =
-                    htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
-                            HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
-                        @Override
-                        public Long call(MetaDataService instance) throws IOException {
-                            ServerRpcController controller = new ServerRpcController();
-                            BlockingRpcCallback<ClearCacheResponse> rpcCallback =
-                                    new BlockingRpcCallback<ClearCacheResponse>();
-                            ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
-                            builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                            instance.clearCache(controller, builder.build(), rpcCallback);
-                            if(controller.getFailedOn() != null) {
-                                throw controller.getFailedOn();
+                        htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW,
+                                HConstants.EMPTY_END_ROW, new Batch.Call<MetaDataService, Long>() {
+                            @Override
+                            public Long call(MetaDataService instance) throws IOException {
+                                ServerRpcController controller = new ServerRpcController();
+                                BlockingRpcCallback<ClearCacheResponse> rpcCallback =
+                                        new BlockingRpcCallback<ClearCacheResponse>();
+                                ClearCacheRequest.Builder builder = ClearCacheRequest.newBuilder();
+                                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                                instance.clearCache(controller, builder.build(), rpcCallback);
+                                if(controller.getFailedOn() != null) {
+                                    throw controller.getFailedOn();
+                                }
+                                return rpcCallback.get().getUnfreedBytes();
                             }
-                            return rpcCallback.get().getUnfreedBytes();
-                        }
-                      });
+                        });
                 long unfreedBytes = 0;
                 for (Map.Entry<byte[],Long> result : results.entrySet()) {
                     if (result.getValue() != null) {
@@ -3016,7 +3042,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             // restore the interrupt status
             Thread.currentThread().interrupt();
             throw new SQLExceptionInfo.Builder(SQLExceptionCode.INTERRUPTED_EXCEPTION).setRootCause(e).build()
-                    .buildException();
+            .buildException();
         } finally {
             Closeables.closeQuietly(admin);
         }
@@ -3253,21 +3279,21 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             try {
                 htable.coprocessorService(MetaDataService.class, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
                         new Batch.Call<MetaDataService, ClearTableFromCacheResponse>() {
-                            @Override
-                            public ClearTableFromCacheResponse call(MetaDataService instance) throws IOException {
-                                ServerRpcController controller = new ServerRpcController();
-                                BlockingRpcCallback<ClearTableFromCacheResponse> rpcCallback = new BlockingRpcCallback<ClearTableFromCacheResponse>();
-                                ClearTableFromCacheRequest.Builder builder = ClearTableFromCacheRequest.newBuilder();
-                                builder.setTenantId(ByteStringer.wrap(tenantId));
-                                builder.setTableName(ByteStringer.wrap(tableName));
-                                builder.setSchemaName(ByteStringer.wrap(schemaName));
-                                builder.setClientTimestamp(clientTS);
-                                builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                                instance.clearTableFromCache(controller, builder.build(), rpcCallback);
-                                if (controller.getFailedOn() != null) { throw controller.getFailedOn(); }
-                                return rpcCallback.get();
-                            }
-                        });
+                    @Override
+                    public ClearTableFromCacheResponse call(MetaDataService instance) throws IOException {
+                        ServerRpcController controller = new ServerRpcController();
+                        BlockingRpcCallback<Clea

<TRUNCATED>