Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.2 2d70f55ae -> 0b1f22749


http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 0b80f4d..a4a4124 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -260,8 +260,6 @@ public interface QueryServices extends SQLCloseable {
     
     //currently BASE64 and ASCII is supported
     public static final String UPLOAD_BINARY_DATA_TYPE_ENCODING = 
"phoenix.upload.binaryDataType.encoding";
-    // Toggle for server-written updates to SYSTEM.CATALOG
-    public static final String PHOENIX_ACLS_ENABLED = "phoenix.acls.enabled";
 
     public static final String INDEX_ASYNC_BUILD_ENABLED = 
"phoenix.index.async.build.enabled";
 
@@ -294,9 +292,6 @@ public interface QueryServices extends SQLCloseable {
     //Update Cache Frequency default config attribute
     public static final String DEFAULT_UPDATE_CACHE_FREQUENCY_ATRRIB  = 
"phoenix.default.update.cache.frequency";
 
-    // Whether to enable cost-based-decision in the query optimizer
-    public static final String COST_BASED_OPTIMIZER_ENABLED = 
"phoenix.costbased.optimizer.enabled";
-
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 4d31974..af6a054 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -25,7 +25,6 @@ import static 
org.apache.phoenix.query.QueryServices.CALL_QUEUE_PRODUCER_ATTRIB_
 import static 
org.apache.phoenix.query.QueryServices.CALL_QUEUE_ROUND_ROBIN_ATTRIB;
 import static 
org.apache.phoenix.query.QueryServices.COLLECT_REQUEST_LEVEL_METRICS;
 import static org.apache.phoenix.query.QueryServices.COMMIT_STATS_ASYNC;
-import static 
org.apache.phoenix.query.QueryServices.COST_BASED_OPTIMIZER_ENABLED;
 import static org.apache.phoenix.query.QueryServices.DATE_FORMAT_ATTRIB;
 import static 
org.apache.phoenix.query.QueryServices.DATE_FORMAT_TIMEZONE_ATTRIB;
 import static 
org.apache.phoenix.query.QueryServices.DELAY_FOR_SCHEMA_UPDATE_CHECK;
@@ -59,7 +58,6 @@ import static 
org.apache.phoenix.query.QueryServices.MAX_TENANT_MEMORY_PERC_ATTR
 import static 
org.apache.phoenix.query.QueryServices.MIN_STATS_UPDATE_FREQ_MS_ATTRIB;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static 
org.apache.phoenix.query.QueryServices.NUM_RETRIES_FOR_SCHEMA_UPDATE_CHECK;
-import static org.apache.phoenix.query.QueryServices.PHOENIX_ACLS_ENABLED;
 import static 
org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_CLUSTER_BASE_PATH;
 import static 
org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_LOADBALANCER_ENABLED;
 import static 
org.apache.phoenix.query.QueryServices.PHOENIX_QUERY_SERVER_SERVICE_NAME;
@@ -318,9 +316,6 @@ public class QueryServicesOptions {
     public static final int DEFAULT_CLIENT_CONNECTION_MAX_ALLOWED_CONNECTIONS 
= 0;
     public static final boolean DEFAULT_STATS_COLLECTION_ENABLED = true;
     public static final boolean DEFAULT_USE_STATS_FOR_PARALLELIZATION = true;
-    
-    //Security defaults
-    public static final boolean DEFAULT_PHOENIX_ACLS_ENABLED = false;
 
     //default update cache frequency
     public static final int DEFAULT_UPDATE_CACHE_FREQUENCY = 0;
@@ -342,8 +337,6 @@ public class QueryServicesOptions {
     // RS -> RS calls for upsert select statements are disabled by default
     public static final boolean DEFAULT_ENABLE_SERVER_UPSERT_SELECT = false;
 
-    public static final boolean DEFAULT_COST_BASED_OPTIMIZER_ENABLED = false;
-
     private final Configuration config;
 
     private QueryServicesOptions(Configuration config) {
@@ -420,10 +413,7 @@ public class QueryServicesOptions {
             .setIfUnset(TRACING_BATCH_SIZE, DEFAULT_TRACING_BATCH_SIZE)
             .setIfUnset(TRACING_THREAD_POOL_SIZE, 
DEFAULT_TRACING_THREAD_POOL_SIZE)
             .setIfUnset(STATS_COLLECTION_ENABLED, 
DEFAULT_STATS_COLLECTION_ENABLED)
-            .setIfUnset(USE_STATS_FOR_PARALLELIZATION, 
DEFAULT_USE_STATS_FOR_PARALLELIZATION)
-            .setIfUnset(COST_BASED_OPTIMIZER_ENABLED, 
DEFAULT_COST_BASED_OPTIMIZER_ENABLED)
-            .setIfUnset(UPLOAD_BINARY_DATA_TYPE_ENCODING, 
DEFAULT_UPLOAD_BINARY_DATA_TYPE_ENCODING)
-            .setIfUnset(PHOENIX_ACLS_ENABLED,  DEFAULT_PHOENIX_ACLS_ENABLED);
+            .setIfUnset(USE_STATS_FOR_PARALLELIZATION, 
DEFAULT_USE_STATS_FOR_PARALLELIZATION);
         // HBase sets this to 1, so we reset it to something more appropriate.
         // Hopefully HBase will change this, because we can't know if a user 
set
         // it to 1, so we'll change it.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/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 5ec5ac3..338b325 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
@@ -113,7 +113,6 @@ import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Types;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
@@ -131,16 +130,11 @@ import java.util.Set;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.client.ClusterConnection;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HConnection;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.security.AccessDeniedException;
-import org.apache.hadoop.hbase.security.access.AccessControlClient;
-import org.apache.hadoop.hbase.security.access.Permission;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.ColumnResolver;
@@ -171,7 +165,6 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.AddColumnStatement;
 import org.apache.phoenix.parse.AlterIndexStatement;
-import org.apache.phoenix.parse.ChangePermsStatement;
 import org.apache.phoenix.parse.CloseStatement;
 import org.apache.phoenix.parse.ColumnDef;
 import org.apache.phoenix.parse.ColumnDefInPkConstraint;
@@ -236,7 +229,6 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
-import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TransactionUtil;
 import org.apache.phoenix.util.UpgradeUtil;
@@ -3007,21 +2999,6 @@ public class MetaDataClient {
         return mutationCode;
     }
 
-    private long incrementTableSeqNum(PTable table, PTableType expectedType, 
int columnCountDelta, MetaPropertiesEvaluated metaPropertiesEvaluated)
-            throws SQLException {
-        return incrementTableSeqNum(table, expectedType, columnCountDelta,
-                metaPropertiesEvaluated.getIsTransactional(),
-                metaPropertiesEvaluated.getUpdateCacheFrequency(),
-                metaPropertiesEvaluated.getIsImmutableRows(),
-                metaPropertiesEvaluated.getDisableWAL(),
-                metaPropertiesEvaluated.getMultiTenant(),
-                metaPropertiesEvaluated.getStoreNulls(),
-                metaPropertiesEvaluated.getGuidePostWidth(),
-                metaPropertiesEvaluated.getAppendOnlySchema(),
-                metaPropertiesEvaluated.getImmutableStorageScheme(),
-                metaPropertiesEvaluated.getUseStatsForParallelization());
-    }
-
     private  long incrementTableSeqNum(PTable table, PTableType expectedType, 
int columnCountDelta, Boolean isTransactional, Long updateCacheFrequency) 
throws SQLException {
         return incrementTableSeqNum(table, expectedType, columnCountDelta, 
isTransactional, updateCacheFrequency, null, null, null, null, -1L, null, null, 
null);
     }
@@ -3153,7 +3130,18 @@ public class MetaDataClient {
             PName tenantId = connection.getTenantId();
             String schemaName = table.getSchemaName().getString();
             String tableName = table.getTableName().getString();
-
+            Boolean isImmutableRowsProp = null;
+            Boolean multiTenantProp = null;
+            Boolean disableWALProp = null;
+            Boolean storeNullsProp = null;
+            Boolean isTransactionalProp = null;
+            Long updateCacheFrequencyProp = null;
+            Boolean appendOnlySchemaProp = null;
+            Long guidePostWidth = -1L;
+            ImmutableStorageScheme immutableStorageSchemeProp = null;
+            Boolean useStatsForParallelizationProp = null;
+
+            Map<String, List<Pair<String, Object>>> properties = new 
HashMap<>(stmtProperties.size());
             List<ColumnDef> columnDefs = null;
             if (table.isAppendOnlySchema()) {
                 // only make the rpc if we are adding new columns
@@ -3189,14 +3177,48 @@ public class MetaDataClient {
             else {
                 columnDefs = origColumnDefs == null ? 
Collections.<ColumnDef>emptyList() : origColumnDefs;
             }
-
+            for (String family : stmtProperties.keySet()) {
+                List<Pair<String, Object>> origPropsList = 
stmtProperties.get(family);
+                List<Pair<String, Object>> propsList = 
Lists.newArrayListWithExpectedSize(origPropsList.size());
+                for (Pair<String, Object> prop : origPropsList) {
+                    String propName = prop.getFirst();
+                    if (TableProperty.isPhoenixTableProperty(propName)) {
+                        TableProperty tableProp = 
TableProperty.valueOf(propName);
+                        tableProp.validate(true, 
!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
+                        Object value = tableProp.getValue(prop.getSecond());
+                        if 
(propName.equals(PTable.IS_IMMUTABLE_ROWS_PROP_NAME)) {
+                            isImmutableRowsProp = (Boolean)value;
+                        } else if 
(propName.equals(PhoenixDatabaseMetaData.MULTI_TENANT)) {
+                            multiTenantProp = (Boolean)value;
+                        } else if (propName.equals(DISABLE_WAL)) {
+                            disableWALProp = (Boolean)value;
+                        } else if (propName.equals(STORE_NULLS)) {
+                            storeNullsProp = (Boolean)value;
+                        } else if (propName.equals(TRANSACTIONAL)) {
+                            isTransactionalProp = (Boolean)value;
+                        } else if (propName.equals(UPDATE_CACHE_FREQUENCY)) {
+                            updateCacheFrequencyProp = (Long)value;
+                        } else if (propName.equals(GUIDE_POSTS_WIDTH)) {
+                            guidePostWidth = (Long)value;
+                        } else if (propName.equals(APPEND_ONLY_SCHEMA)) {
+                            appendOnlySchemaProp = (Boolean) value;
+                        } else if 
(propName.equalsIgnoreCase(IMMUTABLE_STORAGE_SCHEME)) {
+                            immutableStorageSchemeProp = 
(ImmutableStorageScheme)value;
+                        } else if 
(propName.equalsIgnoreCase(USE_STATS_FOR_PARALLELIZATION)) {
+                            useStatsForParallelizationProp = (Boolean)value;
+                        }
+                    }
+                    // if removeTableProps is true only add the property if it 
is not a HTable or Phoenix Table property
+                    if (!removeTableProps || 
(!TableProperty.isPhoenixTableProperty(propName) && 
!MetaDataUtil.isHTableProperty(propName))) {
+                        propsList.add(prop);
+                    }
+                }
+                properties.put(family, propsList);
+            }
             boolean retried = false;
             boolean changingPhoenixTableProperty = false;
-            MetaProperties metaProperties = new MetaProperties();
+            boolean nonTxToTx = false;
             while (true) {
-                Map<String, List<Pair<String, Object>>> properties=new 
HashMap<>(stmtProperties.size());;
-                metaProperties = 
loadStmtProperties(stmtProperties,properties,table,removeTableProps);
-
                 ColumnResolver resolver = 
FromCompiler.getResolver(namedTableNode, connection);
                 table = resolver.getTables().get(0).getTable();
                 int nIndexes = table.getIndexes().size();
@@ -3223,11 +3245,108 @@ public class MetaDataClient {
                     
.setColumnName(lastPK.getName().getString()).build().buildException();
                 }
 
+                Boolean isImmutableRows = null;
+                if (isImmutableRowsProp != null) {
+                    if (isImmutableRowsProp.booleanValue() != 
table.isImmutableRows()) {
+                       if (table.getImmutableStorageScheme() != 
ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
+                               throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY)
+                               
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                       }
+                        isImmutableRows = isImmutableRowsProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+                Boolean multiTenant = null;
+                if (multiTenantProp != null) {
+                    if (multiTenantProp.booleanValue() != 
table.isMultiTenant()) {
+                        multiTenant = multiTenantProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+                Boolean disableWAL = null;
+                if (disableWALProp != null) {
+                    if (disableWALProp.booleanValue() != 
table.isWALDisabled()) {
+                        disableWAL = disableWALProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+                Long updateCacheFrequency = null;
+                if (updateCacheFrequencyProp != null) {
+                    if (updateCacheFrequencyProp.longValue() != 
table.getUpdateCacheFrequency()) {
+                        updateCacheFrequency = updateCacheFrequencyProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+                Boolean appendOnlySchema = null;
+                if (appendOnlySchemaProp !=null) {
+                    if (appendOnlySchemaProp != table.isAppendOnlySchema()) {
+                        appendOnlySchema  = appendOnlySchemaProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+                ImmutableStorageScheme immutableStorageScheme = null;
+                if (immutableStorageSchemeProp!=null) {
+                    if (table.getImmutableStorageScheme() == 
ONE_CELL_PER_COLUMN || 
+                            immutableStorageSchemeProp == ONE_CELL_PER_COLUMN) 
{
+                        throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE)
+                        
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                    }
+                    else if (immutableStorageSchemeProp != 
table.getImmutableStorageScheme()) {
+                        immutableStorageScheme = immutableStorageSchemeProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+            
+                if (guidePostWidth == null || guidePostWidth >= 0) {
+                    changingPhoenixTableProperty = true;
+                }
+                Boolean storeNulls = null;
+                if (storeNullsProp != null) {
+                    if (storeNullsProp.booleanValue() != 
table.getStoreNulls()) {
+                        storeNulls = storeNullsProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
+                Boolean useStatsForParallelization = null;
+                if (useStatsForParallelizationProp != null
+                        && (table.useStatsForParallelization() == null
+                                || 
(useStatsForParallelizationProp.booleanValue() != table
+                                        .useStatsForParallelization()))) {
+                    useStatsForParallelization = 
useStatsForParallelizationProp;
+                    changingPhoenixTableProperty = true;
+                }
+                Boolean isTransactional = null;
+                if (isTransactionalProp != null) {
+                    if (isTransactionalProp.booleanValue() != 
table.isTransactional()) {
+                        isTransactional = isTransactionalProp;
+                        // We can only go one way: from non transactional to 
transactional
+                        // Going the other way would require rewriting the 
cell timestamps
+                        // and doing a major compaction to get rid of any 
Tephra specific
+                        // delete markers.
+                        if (!isTransactional) {
+                            throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX)
+                            
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                        }
+                        // cannot create a transactional table if transactions 
are disabled
+                        boolean transactionsEnabled = 
connection.getQueryServices().getProps().getBoolean(
+                                QueryServices.TRANSACTIONS_ENABLED,
+                                
QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
+                        if (!transactionsEnabled) {
+                            throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED)
+                            
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                        }
+                        // cannot make a table transactional if it has a row 
timestamp column
+                        if (SchemaUtil.hasRowTimestampColumn(table)) {
+                            throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP)
+                            .setSchemaName(schemaName).setTableName(tableName)
+                            .build().buildException();
+                        }
+                        changingPhoenixTableProperty = true;
+                        nonTxToTx = true;
+                    }
+                }
+                Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || nonTxToTx);
 
-                MetaPropertiesEvaluated metaPropertiesEvaluated = new 
MetaPropertiesEvaluated();
-                changingPhoenixTableProperty = 
evaluateStmtProperties(metaProperties,metaPropertiesEvaluated,table,schemaName,tableName);
-
-                Long timeStamp = TransactionUtil.getTableTimestamp(connection, 
table.isTransactional() || metaProperties.getNonTxToTx());
                 int numPkColumnsAdded = 0;
                 List<PColumn> columns = 
Lists.newArrayListWithExpectedSize(numCols);
                 Set<String> colFamiliesForPColumnsToBeAdded = new 
LinkedHashSet<>();
@@ -3344,7 +3463,7 @@ public class MetaDataClient {
                     // Check that HBase configured properly for mutable 
secondary indexing
                     // if we're changing from an immutable table to a mutable 
table and we
                     // have existing indexes.
-                    if 
(Boolean.FALSE.equals(metaPropertiesEvaluated.getIsImmutableRows()) && 
!table.getIndexes().isEmpty()) {
+                    if (Boolean.FALSE.equals(isImmutableRows) && 
!table.getIndexes().isEmpty()) {
                         int hbaseVersion = 
connection.getQueryServices().getLowestClusterHBaseVersion();
                         if (hbaseVersion < 
PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) {
                             throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES)
@@ -3355,22 +3474,22 @@ public class MetaDataClient {
                             
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
                         }
                     }
-                    if 
(Boolean.TRUE.equals(metaPropertiesEvaluated.getMultiTenant())) {
-                        throwIfInsufficientColumns(schemaName, tableName, 
table.getPKColumns(), table.getBucketNum()!=null, 
metaPropertiesEvaluated.getMultiTenant());
+                    if (Boolean.TRUE.equals(multiTenant)) {
+                        throwIfInsufficientColumns(schemaName, tableName, 
table.getPKColumns(), table.getBucketNum()!=null, multiTenant);
                     }
                 }
 
-                if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || 
metaProperties.getNonTxToTx())) {
+                if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || 
nonTxToTx)) {
                     for (PTable index : table.getIndexes()) {
-                        incrementTableSeqNum(index, index.getType(), 
numPkColumnsAdded, metaProperties.getNonTxToTx() ? Boolean.TRUE : null, 
metaPropertiesEvaluated.getUpdateCacheFrequency());
+                        incrementTableSeqNum(index, index.getType(), 
numPkColumnsAdded, nonTxToTx ? Boolean.TRUE : null, updateCacheFrequency);
                     }
                     
tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
                 
                 if (changingPhoenixTableProperty || columnDefs.size() > 0) {
-                    incrementTableSeqNum(table, tableType, columnDefs.size(), 
metaPropertiesEvaluated);
-
+                    incrementTableSeqNum(table, tableType, columnDefs.size(), 
isTransactional, updateCacheFrequency, isImmutableRows,
+                            disableWAL, multiTenant, storeNulls, 
guidePostWidth, appendOnlySchema, immutableStorageScheme, 
useStatsForParallelization);
                     
tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -3440,10 +3559,10 @@ public class MetaDataClient {
                     // We could update the cache manually then too, it'd just 
be a pain.
                     String fullTableName = SchemaUtil.getTableName(schemaName, 
tableName);
                     long resolvedTimeStamp = 
TransactionUtil.getResolvedTime(connection, result);
-                    if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 
&& ! metaProperties.getNonTxToTx())) {
+                    if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 
&& !nonTxToTx)) {
                         connection.addTable(result.getTable(), 
resolvedTimeStamp);
                         table = result.getTable();
-                    } else if 
(metaPropertiesEvaluated.getUpdateCacheFrequency() != null) {
+                    } 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.
                         connection.removeTable(tenantId, fullTableName, null, 
resolvedTimeStamp);
@@ -3451,7 +3570,7 @@ public class MetaDataClient {
                     // Delete rows in view index if we haven't dropped it 
already
                     // We only need to do this if the multiTenant transitioned 
to false
                     if (table.getType() == PTableType.TABLE
-                            && 
Boolean.FALSE.equals(metaPropertiesEvaluated.getMultiTenant())
+                            && Boolean.FALSE.equals(multiTenant)
                             && MetaDataUtil.hasViewIndexTable(connection, 
table.getPhysicalName())) {
                         connection.setAutoCommit(true);
                         MetaDataUtil.deleteViewIndexSequences(connection, 
table.getPhysicalName(), table.isNamespaceMapped());
@@ -3813,19 +3932,9 @@ public class MetaDataClient {
         boolean wasAutoCommit = connection.getAutoCommit();
         try {
             String dataTableName = statement.getTableName();
+            String schemaName = statement.getTable().getName().getSchemaName();
             String indexName = statement.getTable().getName().getTableName();
             boolean isAsync = statement.isAsync();
-            String tenantId = connection.getTenantId() == null ? null : 
connection.getTenantId().getString();
-            PTable table = FromCompiler.getResolver(statement, 
connection).getTables().get(0).getTable();
-            String schemaName = statement.getTable().getName().getSchemaName();
-            String tableName = table.getTableName().getString();
-
-            Map<String, List<Pair<String, Object>>> properties=new 
HashMap<>(statement.getProps().size());;
-            MetaProperties metaProperties = 
loadStmtProperties(statement.getProps(),properties,table,false);
-
-            MetaPropertiesEvaluated metaPropertiesEvaluated = new 
MetaPropertiesEvaluated();
-            boolean changingPhoenixTableProperty= 
evaluateStmtProperties(metaProperties,metaPropertiesEvaluated,table,schemaName,tableName);
-
             PIndexState newIndexState = statement.getIndexState();
             if (isAsync && newIndexState != PIndexState.REBUILD) { throw new 
SQLExceptionInfo.Builder(
                     SQLExceptionCode.ASYNC_NOT_ALLOWED)
@@ -3863,15 +3972,7 @@ public class MetaDataClient {
             List<Mutation> tableMetadata = 
connection.getMutationState().toMutations(timeStamp).next().getSecond();
             connection.rollback();
 
-
-            if (changingPhoenixTableProperty) {
-                incrementTableSeqNum(table,statement.getTableType(), 0, 
metaPropertiesEvaluated);
-                
tableMetadata.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
-                connection.rollback();
-            }
-
-            MetaDataMutationResult result = 
connection.getQueryServices().updateIndexState(tableMetadata, dataTableName, 
properties, table);
-
+            MetaDataMutationResult result = 
connection.getQueryServices().updateIndexState(tableMetadata, dataTableName);
             MutationCode code = result.getMutationCode();
             if (code == MutationCode.TABLE_NOT_FOUND) {
                 throw new TableNotFoundException(schemaName,indexName);
@@ -3980,10 +4081,8 @@ public class MetaDataClient {
                             
SQLExceptionCode.CREATE_SCHEMA_NOT_ALLOWED).setSchemaName(create.getSchemaName())
                             .build().buildException(); }
             boolean isIfNotExists = create.isIfNotExists();
+            validateSchema(create.getSchemaName());
             PSchema schema = new PSchema(create.getSchemaName());
-            // Use SchemaName from PSchema object to get the normalized 
SchemaName
-            // See PHOENIX-4424 for details
-            validateSchema(schema.getSchemaName());
             connection.setAutoCommit(false);
             List<Mutation> schemaMutations;
 
@@ -4018,7 +4117,7 @@ public class MetaDataClient {
 
     private void validateSchema(String schemaName) throws SQLException {
         if (SchemaUtil.NOT_ALLOWED_SCHEMA_LIST.contains(
-                schemaName)) { throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
+                schemaName.toUpperCase())) { throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.SCHEMA_NOT_ALLOWED)
                 .setSchemaName(schemaName).build().buildException(); }
     }
 
@@ -4069,481 +4168,4 @@ public class MetaDataClient {
         }
         return new MutationState(0, 0, connection);
     }
-
-    /**
-     * GRANT/REVOKE statements use this method to update HBase acl's
-     * Perms can be changed at Schema, Table or User level
-     * @throws SQLException
-     */
-    public MutationState changePermissions(ChangePermsStatement 
changePermsStatement) throws SQLException {
-
-        logger.info(changePermsStatement.toString());
-
-        try(HBaseAdmin admin = connection.getQueryServices().getAdmin()) {
-            ClusterConnection clusterConnection = (ClusterConnection) 
admin.getConnection();
-
-            if (changePermsStatement.getSchemaName() != null) {
-                // SYSTEM.CATALOG doesn't have any entry for "default" HBase 
namespace, hence we will bypass the check
-                
if(!changePermsStatement.getSchemaName().equals(SchemaUtil.SCHEMA_FOR_DEFAULT_NAMESPACE))
 {
-                    
FromCompiler.getResolverForSchema(changePermsStatement.getSchemaName(), 
connection);
-                }
-
-                changePermsOnSchema(clusterConnection, changePermsStatement);
-            } else if (changePermsStatement.getTableName() != null) {
-                PTable inputTable = PhoenixRuntime.getTable(connection,
-                        
SchemaUtil.normalizeFullTableName(changePermsStatement.getTableName().toString()));
-                if (!(PTableType.TABLE.equals(inputTable.getType()) || 
PTableType.SYSTEM.equals(inputTable.getType()))) {
-                    throw new AccessDeniedException("Cannot GRANT or REVOKE 
permissions on INDEX TABLES or VIEWS");
-                }
-
-                // Changing perms on base table and update the perms for 
global and view indexes
-                // Views and local indexes are not physical tables and hence 
update perms is not needed
-                changePermsOnTables(clusterConnection, admin, 
changePermsStatement, inputTable);
-            } else {
-
-                // User can be given perms at the global level
-                changePermsOnUser(clusterConnection, changePermsStatement);
-            }
-
-        } catch (SQLException e) {
-            // Bubble up the SQL Exception
-            throw e;
-        } catch (Throwable throwable) {
-            // To change perms, the user must have ADMIN perms on that scope, 
otherwise it throws ADE
-            // Wrap around ADE and other exceptions to PhoenixIOException
-            throw ServerUtil.parseServerException(throwable);
-        }
-
-        return new MutationState(0, 0, connection);
-    }
-
-    private void changePermsOnSchema(ClusterConnection clusterConnection, 
ChangePermsStatement changePermsStatement) throws Throwable {
-        if(changePermsStatement.isGrantStatement()) {
-            AccessControlClient.grant(clusterConnection, 
changePermsStatement.getSchemaName(), changePermsStatement.getName(), 
changePermsStatement.getPermsList());
-        } else {
-            AccessControlClient.revoke(clusterConnection, 
changePermsStatement.getSchemaName(), changePermsStatement.getName(), 
Permission.Action.values());
-        }
-    }
-
-    private void changePermsOnTables(ClusterConnection clusterConnection, 
HBaseAdmin admin, ChangePermsStatement changePermsStatement, PTable inputTable) 
throws Throwable {
-
-        org.apache.hadoop.hbase.TableName tableName = 
SchemaUtil.getPhysicalTableName
-                (inputTable.getPhysicalName().getBytes(), 
inputTable.isNamespaceMapped());
-
-        changePermsOnTable(clusterConnection, changePermsStatement, tableName);
-
-        boolean schemaInconsistency = false;
-        List<PTable> inconsistentTables = null;
-
-        for(PTable indexTable : inputTable.getIndexes()) {
-            // Local Indexes don't correspond to new physical table, they are 
just stored in separate CF of base table.
-            if(indexTable.getIndexType().equals(IndexType.LOCAL)) {
-                continue;
-            }
-            if (inputTable.isNamespaceMapped() != 
indexTable.isNamespaceMapped()) {
-                schemaInconsistency = true;
-                if(inconsistentTables == null) {
-                    inconsistentTables = new ArrayList<>();
-                }
-                inconsistentTables.add(indexTable);
-                continue;
-            }
-            logger.info("Updating permissions for Index Table: " +
-                    indexTable.getName() + " Base Table: " + 
inputTable.getName());
-            tableName = 
SchemaUtil.getPhysicalTableName(indexTable.getPhysicalName().getBytes(), 
indexTable.isNamespaceMapped());
-            changePermsOnTable(clusterConnection, changePermsStatement, 
tableName);
-        }
-
-        if(schemaInconsistency) {
-            for(PTable table : inconsistentTables) {
-                logger.error("Fail to propagate permissions to Index Table: " 
+ table.getName());
-            }
-            throw new 
TablesNotInSyncException(inputTable.getTableName().getString(),
-                    inconsistentTables.get(0).getTableName().getString(), 
"Namespace properties");
-        }
-
-        // There will be only a single View Index Table for all the indexes 
created on views
-        byte[] viewIndexTableBytes = 
MetaDataUtil.getViewIndexPhysicalName(inputTable.getPhysicalName().getBytes());
-        tableName = 
org.apache.hadoop.hbase.TableName.valueOf(viewIndexTableBytes);
-        boolean viewIndexTableExists = admin.tableExists(tableName);
-        if(viewIndexTableExists) {
-            logger.info("Updating permissions for View Index Table: " +
-                    Bytes.toString(viewIndexTableBytes) + " Base Table: " + 
inputTable.getName());
-            changePermsOnTable(clusterConnection, changePermsStatement, 
tableName);
-        } else {
-            if(inputTable.isMultiTenant()) {
-                logger.error("View Index Table not found for MultiTenant 
Table: " + inputTable.getName());
-                logger.error("Fail to propagate permissions to view Index 
Table: " + tableName.getNameAsString());
-                throw new 
TablesNotInSyncException(inputTable.getTableName().getString(),
-                        Bytes.toString(viewIndexTableBytes), " View Index 
table should exist for MultiTenant tables");
-            }
-        }
-    }
-
-    private void changePermsOnTable(ClusterConnection clusterConnection, 
ChangePermsStatement changePermsStatement, org.apache.hadoop.hbase.TableName 
tableName)
-            throws Throwable {
-        if(changePermsStatement.isGrantStatement()) {
-            AccessControlClient.grant(clusterConnection, tableName, 
changePermsStatement.getName(),
-                    null, null, changePermsStatement.getPermsList());
-        } else {
-            AccessControlClient.revoke(clusterConnection, tableName, 
changePermsStatement.getName(),
-                    null, null, Permission.Action.values());
-        }
-    }
-
-    private void changePermsOnUser(ClusterConnection clusterConnection, 
ChangePermsStatement changePermsStatement)
-            throws Throwable {
-        if(changePermsStatement.isGrantStatement()) {
-            AccessControlClient.grant(clusterConnection, 
changePermsStatement.getName(), changePermsStatement.getPermsList());
-        } else {
-            AccessControlClient.revoke(clusterConnection, 
changePermsStatement.getName(), Permission.Action.values());
-        }
-    }
-
-    private MetaProperties loadStmtProperties(ListMultimap<String, 
Pair<String, Object>> stmtProperties, Map<String, List<Pair<String, Object>>> 
properties, PTable table, boolean removeTableProps)
-            throws SQLException {
-        MetaProperties metaProperties = new MetaProperties();
-        for (String family : stmtProperties.keySet()) {
-            List<Pair<String, Object>> origPropsList = 
stmtProperties.get(family);
-            List<Pair<String, Object>> propsList = 
Lists.newArrayListWithExpectedSize(origPropsList.size());
-            for (Pair<String, Object> prop : origPropsList) {
-                String propName = prop.getFirst();
-                if (TableProperty.isPhoenixTableProperty(propName)) {
-                    TableProperty tableProp = TableProperty.valueOf(propName);
-                    tableProp.validate(true, 
!family.equals(QueryConstants.ALL_FAMILY_PROPERTIES_KEY), table.getType());
-                    Object value = tableProp.getValue(prop.getSecond());
-                    if (propName.equals(PTable.IS_IMMUTABLE_ROWS_PROP_NAME)) {
-                        metaProperties.setImmutableRowsProp((Boolean)value);
-                    } else if 
(propName.equals(PhoenixDatabaseMetaData.MULTI_TENANT)) {
-                        metaProperties.setMultiTenantProp((Boolean)value);
-                    } else if (propName.equals(DISABLE_WAL)) {
-                        metaProperties.setDisableWALProp((Boolean)value);
-                    } else if (propName.equals(STORE_NULLS)) {
-                        metaProperties.setStoreNullsProp((Boolean)value);
-                    } else if (propName.equals(TRANSACTIONAL)) {
-                        metaProperties.setIsTransactionalProp((Boolean)value);
-                    } else if (propName.equals(UPDATE_CACHE_FREQUENCY)) {
-                        
metaProperties.setUpdateCacheFrequencyProp((Long)value);
-                    } else if (propName.equals(GUIDE_POSTS_WIDTH)) {
-                        metaProperties.setGuidePostWidth((Long)value);
-                    } else if (propName.equals(APPEND_ONLY_SCHEMA)) {
-                        metaProperties.setAppendOnlySchemaProp((Boolean) 
value);
-                    } else if 
(propName.equalsIgnoreCase(IMMUTABLE_STORAGE_SCHEME)) {
-                        
metaProperties.setImmutableStorageSchemeProp((ImmutableStorageScheme)value);
-                    } else if 
(propName.equalsIgnoreCase(USE_STATS_FOR_PARALLELIZATION)) {
-                        
metaProperties.setUseStatsForParallelizationProp((Boolean)value);
-                    }
-                }
-                // if removeTableProps is true only add the property if it is 
not a HTable or Phoenix Table property
-                if (!removeTableProps || 
(!TableProperty.isPhoenixTableProperty(propName) && 
!MetaDataUtil.isHTableProperty(propName))) {
-                    propsList.add(prop);
-                }
-            }
-            properties.put(family, propsList);
-        }
-        return metaProperties;
-    }
-
-    private boolean evaluateStmtProperties(MetaProperties metaProperties, 
MetaPropertiesEvaluated metaPropertiesEvaluated, PTable table, String 
schemaName, String tableName)
-            throws SQLException {
-        boolean changingPhoenixTableProperty = false;
-
-        if (metaProperties.getImmutableRowsProp() != null) {
-            if (metaProperties.getImmutableRowsProp().booleanValue() != 
table.isImmutableRows()) {
-                if (table.getImmutableStorageScheme() != 
ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
-                    throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY)
-                            
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                }
-                
metaPropertiesEvaluated.setIsImmutableRows(metaProperties.getImmutableRowsProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getMultiTenantProp() != null) {
-            if (metaProperties.getMultiTenantProp().booleanValue() != 
table.isMultiTenant()) {
-                
metaPropertiesEvaluated.setMultiTenant(metaProperties.getMultiTenantProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getDisableWALProp() != null) {
-            if (metaProperties.getDisableWALProp().booleanValue() != 
table.isWALDisabled()) {
-                
metaPropertiesEvaluated.setDisableWAL(metaProperties.getDisableWALProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getUpdateCacheFrequencyProp() != null) {
-            if (metaProperties.getUpdateCacheFrequencyProp().longValue() != 
table.getUpdateCacheFrequency()) {
-                
metaPropertiesEvaluated.setUpdateCacheFrequency(metaProperties.getUpdateCacheFrequencyProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getAppendOnlySchemaProp() !=null) {
-            if (metaProperties.getAppendOnlySchemaProp() != 
table.isAppendOnlySchema()) {
-                
metaPropertiesEvaluated.setAppendOnlySchema(metaProperties.getAppendOnlySchemaProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getImmutableStorageSchemeProp()!=null) {
-            if (table.getImmutableStorageScheme() == ONE_CELL_PER_COLUMN ||
-                    metaProperties.getImmutableStorageSchemeProp() == 
ONE_CELL_PER_COLUMN) {
-                throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE)
-                        
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
-            }
-            else if (metaProperties.getImmutableStorageSchemeProp() != 
table.getImmutableStorageScheme()) {
-                
metaPropertiesEvaluated.setImmutableStorageScheme(metaProperties.getImmutableStorageSchemeProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getGuidePostWidth() == null || 
metaProperties.getGuidePostWidth() >= 0) {
-            
metaPropertiesEvaluated.setGuidePostWidth(metaProperties.getGuidePostWidth());
-            changingPhoenixTableProperty = true;
-        }
-
-        if (metaProperties.getStoreNullsProp() != null) {
-            if (metaProperties.getStoreNullsProp().booleanValue() != 
table.getStoreNulls()) {
-                
metaPropertiesEvaluated.setStoreNulls(metaProperties.getStoreNullsProp());
-                changingPhoenixTableProperty = true;
-            }
-        }
-
-        if (metaProperties.getUseStatsForParallelizationProp() != null
-                && (table.useStatsForParallelization() == null
-                || 
(metaProperties.getUseStatsForParallelizationProp().booleanValue() != table
-                .useStatsForParallelization()))) {
-            
metaPropertiesEvaluated.setUseStatsForParallelization(metaProperties.getUseStatsForParallelizationProp());
-            changingPhoenixTableProperty = true;
-        }
-
-        if (metaProperties.getIsTransactionalProp() != null) {
-            if (metaProperties.getIsTransactionalProp().booleanValue() != 
table.isTransactional()) {
-                
metaPropertiesEvaluated.setIsTransactional(metaProperties.getIsTransactionalProp());
-                // We can only go one way: from non transactional to 
transactional
-                // Going the other way would require rewriting the cell 
timestamps
-                // and doing a major compaction to get rid of any Tephra 
specific
-                // delete markers.
-                if (!metaPropertiesEvaluated.getIsTransactional()) {
-                    throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX)
-                            
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                }
-                // cannot create a transactional table if transactions are 
disabled
-                boolean transactionsEnabled = 
connection.getQueryServices().getProps().getBoolean(
-                        QueryServices.TRANSACTIONS_ENABLED,
-                        QueryServicesOptions.DEFAULT_TRANSACTIONS_ENABLED);
-                if (!transactionsEnabled) {
-                    throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_IF_TXNS_DISABLED)
-                            
.setSchemaName(schemaName).setTableName(tableName).build().buildException();
-                }
-                // cannot make a table transactional if it has a row timestamp 
column
-                if (SchemaUtil.hasRowTimestampColumn(table)) {
-                    throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP)
-                            .setSchemaName(schemaName).setTableName(tableName)
-                            .build().buildException();
-                }
-                changingPhoenixTableProperty = true;
-                metaProperties.setNonTxToTx(true);
-            }
-        }
-        return changingPhoenixTableProperty;
-    }
-
-    class MetaProperties{
-        private Boolean isImmutableRowsProp = null;
-        private Boolean multiTenantProp = null;
-        private Boolean disableWALProp = null;
-        private Boolean storeNullsProp = null;
-        private Boolean isTransactionalProp = null;
-        private Long updateCacheFrequencyProp = null;
-        private Boolean appendOnlySchemaProp = null;
-        private Long guidePostWidth = -1L;
-        private ImmutableStorageScheme immutableStorageSchemeProp = null;
-        private Boolean useStatsForParallelizationProp = null;
-        private boolean nonTxToTx = false;
-
-        public Boolean getImmutableRowsProp() {
-            return isImmutableRowsProp;
-        }
-
-        public void setImmutableRowsProp(Boolean isImmutableRowsProp) {
-            this.isImmutableRowsProp = isImmutableRowsProp;
-        }
-
-        public Boolean getMultiTenantProp() {
-            return multiTenantProp;
-        }
-
-        public void setMultiTenantProp(Boolean multiTenantProp) {
-            this.multiTenantProp = multiTenantProp;
-        }
-
-        public Boolean getDisableWALProp() {
-            return disableWALProp;
-        }
-
-        public void setDisableWALProp(Boolean disableWALProp) {
-            this.disableWALProp = disableWALProp;
-        }
-
-        public Boolean getStoreNullsProp() {
-            return storeNullsProp;
-        }
-
-        public void setStoreNullsProp(Boolean storeNullsProp) {
-            this.storeNullsProp = storeNullsProp;
-        }
-
-        public Boolean getIsTransactionalProp() {
-            return isTransactionalProp;
-        }
-
-        public void setIsTransactionalProp(Boolean isTransactionalProp) {
-            this.isTransactionalProp = isTransactionalProp;
-        }
-
-        public Long getUpdateCacheFrequencyProp() {
-            return updateCacheFrequencyProp;
-        }
-
-        public void setUpdateCacheFrequencyProp(Long updateCacheFrequencyProp) 
{
-            this.updateCacheFrequencyProp = updateCacheFrequencyProp;
-        }
-
-        public Boolean getAppendOnlySchemaProp() {
-            return appendOnlySchemaProp;
-        }
-
-        public void setAppendOnlySchemaProp(Boolean appendOnlySchemaProp) {
-            this.appendOnlySchemaProp = appendOnlySchemaProp;
-        }
-
-        public Long getGuidePostWidth() {
-            return guidePostWidth;
-        }
-
-        public void setGuidePostWidth(Long guidePostWidth) {
-            this.guidePostWidth = guidePostWidth;
-        }
-
-        public ImmutableStorageScheme getImmutableStorageSchemeProp() {
-            return immutableStorageSchemeProp;
-        }
-
-        public void setImmutableStorageSchemeProp(
-                ImmutableStorageScheme immutableStorageSchemeProp) {
-            this.immutableStorageSchemeProp = immutableStorageSchemeProp;
-        }
-
-        public Boolean getUseStatsForParallelizationProp() {
-            return useStatsForParallelizationProp;
-        }
-
-        public void setUseStatsForParallelizationProp(Boolean 
useStatsForParallelizationProp) {
-            this.useStatsForParallelizationProp = 
useStatsForParallelizationProp;
-        }
-
-        public boolean getNonTxToTx() {
-            return nonTxToTx;
-        }
-
-        public void setNonTxToTx(boolean nonTxToTx) {
-            this.nonTxToTx = nonTxToTx;
-        }
-    }
-
-    class MetaPropertiesEvaluated{
-        private Boolean isImmutableRows;
-        private Boolean multiTenant = null;
-        private Boolean disableWAL = null;
-        private Long updateCacheFrequency = null;
-        private Boolean appendOnlySchema = null;
-        private Long guidePostWidth = -1L;
-        private ImmutableStorageScheme immutableStorageScheme = null;
-        private Boolean storeNulls = null;
-        private Boolean useStatsForParallelization = null;
-        private Boolean isTransactional = null;
-
-        public Boolean getIsImmutableRows() {
-            return isImmutableRows;
-        }
-
-        public void setIsImmutableRows(Boolean isImmutableRows) {
-            this.isImmutableRows = isImmutableRows;
-        }
-
-        public Boolean getMultiTenant() {
-            return multiTenant;
-        }
-
-        public void setMultiTenant(Boolean multiTenant) {
-            this.multiTenant = multiTenant;
-        }
-
-        public Boolean getDisableWAL() {
-            return disableWAL;
-        }
-
-        public void setDisableWAL(Boolean disableWAL) {
-            this.disableWAL = disableWAL;
-        }
-
-        public Long getUpdateCacheFrequency() {
-            return updateCacheFrequency;
-        }
-
-        public void setUpdateCacheFrequency(Long updateCacheFrequency) {
-            this.updateCacheFrequency = updateCacheFrequency;
-        }
-
-        public Boolean getAppendOnlySchema() {
-            return appendOnlySchema;
-        }
-
-        public void setAppendOnlySchema(Boolean appendOnlySchema) {
-            this.appendOnlySchema = appendOnlySchema;
-        }
-
-        public Long getGuidePostWidth() {
-            return guidePostWidth;
-        }
-
-        public void setGuidePostWidth(Long guidePostWidth) {
-            this.guidePostWidth = guidePostWidth;
-        }
-
-        public ImmutableStorageScheme getImmutableStorageScheme() {
-            return immutableStorageScheme;
-        }
-
-        public void setImmutableStorageScheme(ImmutableStorageScheme 
immutableStorageScheme) {
-            this.immutableStorageScheme = immutableStorageScheme;
-        }
-
-        public Boolean getStoreNulls() {
-            return storeNulls;
-        }
-
-        public void setStoreNulls(Boolean storeNulls) {
-            this.storeNulls = storeNulls;
-        }
-
-        public Boolean getUseStatsForParallelization() {
-            return useStatsForParallelization;
-        }
-
-        public void setUseStatsForParallelization(Boolean 
useStatsForParallelization) {
-            this.useStatsForParallelization = useStatsForParallelization;
-        }
-
-        public Boolean getIsTransactional() {
-            return isTransactional;
-        }
-
-        public void setIsTransactional(Boolean isTransactional) {
-            this.isTransactional = isTransactional;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
deleted file mode 100644
index e58df71..0000000
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/TablesNotInSyncException.java
+++ /dev/null
@@ -1,22 +0,0 @@
-package org.apache.phoenix.schema;
-
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.exception.SQLExceptionInfo;
-
-import java.sql.SQLException;
-
-/**
- * Exception to raise when multiple tables differ in specified properties
- * This can happen since Apache Phoenix code doesn't work atomically for many 
parts
- * For example, Base table and index tables are inconsistent in namespace 
mapping
- * OR View Index table doesn't exist for multi-tenant base table
- */
-public class TablesNotInSyncException extends SQLException {
-    private static final long serialVersionUID = 1L;
-    private static SQLExceptionCode code = SQLExceptionCode.TABLES_NOT_IN_SYNC;
-
-    public TablesNotInSyncException(String table1, String table2, String diff) 
{
-        super(new SQLExceptionInfo.Builder(code).setMessage("Table: " + table1 
+ " and Table: " + table2 + " differ in " + diff).build().toString(), 
code.getSQLState(), code.getErrorCode());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
index 8956862..3ae3183 100644
--- 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
+++ 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/StatisticsWriter.java
@@ -25,7 +25,6 @@ import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.EOFException;
 import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
 import java.sql.Date;
 import java.util.ArrayList;
 import java.util.List;
@@ -47,7 +46,6 @@ import 
org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.Mut
 import 
org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
 import 
org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MutateRowsRequest;
 import org.apache.hadoop.hbase.regionserver.Region;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -210,31 +208,23 @@ public class StatisticsWriter implements Closeable {
         }
     }
 
-    public void commitStats(final List<Mutation> mutations, final 
StatisticsCollector statsCollector)
-            throws IOException {
-        User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
-            @Override
-            public Void run() throws Exception {
-                commitLastStatsUpdatedTime(statsCollector);
-                if (mutations.size() > 0) {
-                    byte[] row = mutations.get(0).getRow();
-                    MutateRowsRequest.Builder mrmBuilder = 
MutateRowsRequest.newBuilder();
-                    for (Mutation m : mutations) {
-                        
mrmBuilder.addMutationRequest(ProtobufUtil.toMutation(getMutationType(m), m));
-                    }
-                    MutateRowsRequest mrm = mrmBuilder.build();
-                    CoprocessorRpcChannel channel = 
statsWriterTable.coprocessorService(row);
-                    MultiRowMutationService.BlockingInterface service = 
MultiRowMutationService
-                            .newBlockingStub(channel);
-                    try {
-                        service.mutateRows(null, mrm);
-                    } catch (ServiceException ex) {
-                        ProtobufUtil.toIOException(ex);
-                    }
-                }
-                return null;
+    public void commitStats(List<Mutation> mutations, StatisticsCollector 
statsCollector) throws IOException {
+        commitLastStatsUpdatedTime(statsCollector);
+        if (mutations.size() > 0) {
+            byte[] row = mutations.get(0).getRow();
+            MutateRowsRequest.Builder mrmBuilder = 
MutateRowsRequest.newBuilder();
+            for (Mutation m : mutations) {
+                
mrmBuilder.addMutationRequest(ProtobufUtil.toMutation(getMutationType(m), m));
             }
-        });
+            MutateRowsRequest mrm = mrmBuilder.build();
+            CoprocessorRpcChannel channel = 
statsWriterTable.coprocessorService(row);
+            MultiRowMutationService.BlockingInterface service = 
MultiRowMutationService.newBlockingStub(channel);
+            try {
+                service.mutateRows(null, mrm);
+            } catch (ServiceException ex) {
+                ProtobufUtil.toIOException(ex);
+            }
+        }
     }
 
     private Put getLastStatsUpdatedTimePut(long timeStamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java
deleted file mode 100644
index 1d4b8e0..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/CostUtil.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.util;
-
-import org.apache.phoenix.compile.GroupByCompiler.GroupBy;
-import org.apache.phoenix.optimize.Cost;
-import org.apache.phoenix.query.QueryServices;
-
-/**
- * Utilities for computing costs.
- *
- * Some of the methods here should eventually be replaced by a metadata 
framework which
- * estimates output metrics for each QueryPlan or operation, e.g. row count, 
byte count,
- * etc.
- */
-public class CostUtil {
-
-    // An estimate of the ratio of result data from group-by against the input 
data.
-    private final static double GROUPING_FACTOR = 0.1;
-
-    // Io operations conducted in intermediate evaluations like sorting or 
aggregation
-    // should be counted twice since they usually involve both read and write.
-    private final static double IO_COST_MULTIPLIER = 2.0;
-
-    /**
-     * Estimate the number of output bytes of an aggregate.
-     * @param byteCount the number of input bytes
-     * @param groupBy the compiled GroupBy object
-     * @param aggregatorsSize the byte size of aggregators
-     * @return the output byte count
-     */
-    public static double estimateAggregateOutputBytes(
-            double byteCount, GroupBy groupBy, int aggregatorsSize) {
-        if (groupBy.isUngroupedAggregate()) {
-            return aggregatorsSize;
-        }
-        return byteCount * GROUPING_FACTOR;
-    }
-
-    /**
-     * Estimate the cost of an aggregate.
-     * @param byteCount the number of input bytes
-     * @param groupBy the compiled GroupBy object
-     * @param aggregatorsSize the byte size of aggregators
-     * @param parallelLevel number of parallel workers or threads
-     * @return the cost
-     */
-    public static Cost estimateAggregateCost(
-            double byteCount, GroupBy groupBy, int aggregatorsSize, int 
parallelLevel) {
-        double outputBytes = estimateAggregateOutputBytes(byteCount, groupBy, 
aggregatorsSize);
-        double orderedFactor = groupBy.isOrderPreserving() ? 0.2 : 1.0;
-        return new Cost(0, 0, outputBytes * orderedFactor * IO_COST_MULTIPLIER 
/ parallelLevel);
-    }
-
-    /**
-     * Estimate the cost of an order-by
-     * @param byteCount the number of input bytes
-     * @param parallelLevel number of parallel workers or threads
-     * @return the cost
-     */
-    public static Cost estimateOrderByCost(double byteCount, int 
parallelLevel) {
-        return new Cost(0, 0, byteCount * IO_COST_MULTIPLIER / parallelLevel);
-    }
-
-    /**
-     * Estimate the parallel level of an operation
-     * @param runningOnServer if the operation will be running on server side
-     * @param services the QueryServices object
-     * @return the parallel level
-     */
-    public static int estimateParallelLevel(boolean runningOnServer, 
QueryServices services) {
-        // TODO currently return constants for simplicity, should derive from 
cluster config.
-        return runningOnServer ? 10 : 1;
-    }
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 74f91b4..b23ea1b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -73,7 +73,7 @@ import 
org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import 
org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
+import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -296,7 +296,7 @@ public class IndexUtil {
     }
     
     public static List<Mutation> generateIndexData(final PTable table, PTable 
index,
-            final MultiRowMutationState multiRowMutationState, List<Mutation> 
dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
+            final Map<ImmutableBytesPtr, RowMutationState> valuesMap, 
List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, 
PhoenixConnection connection)
             throws SQLException {
         try {
                final ImmutableBytesPtr ptr = new ImmutableBytesPtr();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index df6a349..2dfe1b9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -30,11 +30,14 @@ import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.phoenix.execute.MutationState.MultiRowMutationState;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
 
 /**
  * 
@@ -184,26 +187,47 @@ public class KeyValueUtil {
     }
 
     /**
-     * Estimates the size of rows stored in RowMutationState (in memory)
+     * Estimates the storage size of a row
      * @param mutations map from table to row to RowMutationState
      * @return estimated row size
      */
     public static long
-            getEstimatedRowMutationSize(Map<TableRef, MultiRowMutationState> 
tableMutationMap) {
+            getEstimatedRowSize(TableRef tableRef, Map<ImmutableBytesPtr, 
RowMutationState> mutations) {
         long size = 0;
-        // iterate over table
-        for (Entry<TableRef, MultiRowMutationState> tableEntry : 
tableMutationMap.entrySet()) {
-            // iterate over rows
-            for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : 
tableEntry.getValue().entrySet()) {
-                size += calculateRowMutationSize(rowEntry);
+        PTable table = tableRef.getTable();
+        // iterate over rows
+        for (Entry<ImmutableBytesPtr, RowMutationState> rowEntry : 
mutations.entrySet()) {
+            int rowLength = rowEntry.getKey().getLength();
+            Map<PColumn, byte[]> colValueMap = 
rowEntry.getValue().getColumnValues();
+            switch (table.getImmutableStorageScheme()) {
+            case ONE_CELL_PER_COLUMN:
+                // iterate over columns
+                for (Entry<PColumn, byte[]> colValueEntry : 
colValueMap.entrySet()) {
+                    PColumn pColumn = colValueEntry.getKey();
+                    size +=
+                            KeyValue.getKeyValueDataStructureSize(rowLength,
+                                pColumn.getFamilyName().getBytes().length,
+                                pColumn.getColumnQualifierBytes().length,
+                                colValueEntry.getValue().length);
+                }
+                break;
+            case SINGLE_CELL_ARRAY_WITH_OFFSETS:
+                // we store all the column values in a single key value that 
contains all the
+                // column values followed by an offset array
+                size +=
+                        PArrayDataTypeEncoder.getEstimatedByteSize(table, 
rowLength,
+                            colValueMap);
+                break;
             }
+            // count the empty key value
+            Pair<byte[], byte[]> emptyKeyValueInfo =
+                    EncodedColumnsUtil.getEmptyKeyValueInfo(table);
+            size +=
+                    KeyValue.getKeyValueDataStructureSize(rowLength,
+                        SchemaUtil.getEmptyColumnFamilyPtr(table).getLength(),
+                        emptyKeyValueInfo.getFirst().length,
+                        emptyKeyValueInfo.getSecond().length);
         }
         return size;
     }
-
-    private static long calculateRowMutationSize(Entry<ImmutableBytesPtr, 
RowMutationState> rowEntry) {
-        int rowLength = rowEntry.getKey().getLength();
-        long colValuesLength = rowEntry.getValue().calculateEstimatedSize();
-        return (rowLength + colValuesLength);
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 2a0c8f0..502ef37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -59,7 +59,6 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SequenceKey;
@@ -227,16 +226,6 @@ public class MetaDataUtil {
         }
         return null;
     }
-
-    public static boolean isNameSpaceMapped(List<Mutation> tableMetaData, 
KeyValueBuilder builder,
-            ImmutableBytesWritable value) {
-        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
-            PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED_BYTES, builder, 
value)) {
-            return 
(boolean)PBoolean.INSTANCE.toObject(ByteUtil.copyKeyBytesIfNecessary(value));
-        }
-        return false;
-    }
-
     
     public static long getParentSequenceNumber(List<Mutation> tableMetaData) {
         return getSequenceNumber(getParentTableHeaderRow(tableMetaData));
@@ -681,11 +670,4 @@ public class MetaDataUtil {
         byte[] physicalTableName = 
Bytes.toBytes(SchemaUtil.getTableNameFromFullName(view.getPhysicalName().getString()));
         return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, 
physicalTableSchemaName, physicalTableName);
     }
-    
-    public static IndexType getIndexType(List<Mutation> tableMetaData, 
KeyValueBuilder builder,
-            ImmutableBytesWritable value) {
-        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData), 
PhoenixDatabaseMetaData.INDEX_TYPE_BYTES, builder,
-                value)) { return 
IndexType.fromSerializedValue(value.get()[value.getOffset()]); }
-        return null;
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
index 685b8cb..f6eb5c5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PropertiesUtil.java
@@ -17,13 +17,10 @@
  */
 package org.apache.phoenix.util;
 
-import java.util.Collections;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
-import java.util.Set;
-
 import org.apache.hadoop.conf.Configuration;
 
 public class PropertiesUtil {
@@ -53,17 +50,13 @@ public class PropertiesUtil {
      *         properties contained in conf
      */
     public static Properties combineProperties(Properties props, final 
Configuration conf) {
-        return combineProperties(props, conf, Collections.<String>emptySet());
-    }
-    
-    public static Properties combineProperties(Properties props, final 
Configuration conf, Set<String> withoutTheseProps) {
         Iterator<Map.Entry<String, String>> iterator = conf.iterator();
         Properties copy = deepCopy(props);
         if (iterator != null) {
             while (iterator.hasNext()) {
                 Map.Entry<String, String> entry = iterator.next();
                 // set the property from config only if props doesn't have it 
already
-                if (copy.getProperty(entry.getKey()) == null && 
!withoutTheseProps.contains(entry.getKey())) {
+                if (copy.getProperty(entry.getKey()) == null) {
                     copy.setProperty(entry.getKey(), entry.getValue());
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java 
b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 42c2dcb..51f6ff9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -20,11 +20,9 @@ package org.apache.phoenix.util;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Strings.isNullOrEmpty;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_NAMESPACE_MAPPED_BYTES;
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_NAME_BYTES;
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES;
-import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
@@ -44,10 +42,8 @@ import java.util.TreeSet;
 import javax.annotation.Nullable;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -58,7 +54,6 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
@@ -81,7 +76,6 @@ import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableProperty;
 import org.apache.phoenix.schema.ValueSchema.Field;
-import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
@@ -133,9 +127,8 @@ public class SchemaUtil {
         
     };
     public static final RowKeySchema VAR_BINARY_SCHEMA = new 
RowKeySchemaBuilder(1).addField(VAR_BINARY_DATUM, false, 
SortOrder.getDefault()).build();
-    // See PHOENIX-4424
-    public static final String SCHEMA_FOR_DEFAULT_NAMESPACE = "default";
-    public static final String HBASE_NAMESPACE = "hbase";
+    public static final String SCHEMA_FOR_DEFAULT_NAMESPACE = "DEFAULT";
+    public static final String HBASE_NAMESPACE = "HBASE";
     public static final List<String> NOT_ALLOWED_SCHEMA_LIST = 
Arrays.asList(SCHEMA_FOR_DEFAULT_NAMESPACE,
             HBASE_NAMESPACE);
     
@@ -207,25 +200,7 @@ public class SchemaUtil {
         }
         return name.toUpperCase();
     }
-
-    /**
-     * Normalize a Literal. If literal is surrounded by single quotes,
-     * the quotes are trimmed, else full string is returned
-     * @param literal the parsed LiteralParseNode
-     * @return the normalized literal string
-     */
-    public static String normalizeLiteral(LiteralParseNode literal) {
-        if (literal == null) {
-            return null;
-        }
-        String literalString = literal.toString();
-        if (isEnclosedInSingleQuotes(literalString)) {
-            // Trim the single quotes
-            return literalString.substring(1, literalString.length()-1);
-        }
-        return literalString;
-    }
-
+    
     /**
      * Normalizes the fulltableName . Uses {@linkplain normalizeIdentifier}
      * @param fullTableName
@@ -241,10 +216,6 @@ public class SchemaUtil {
         return normalizedTableName + normalizeIdentifier(tableName);
     }
 
-    public static boolean isEnclosedInSingleQuotes(String name) {
-        return name!=null && name.length() > 0 && name.charAt(0)=='\'';
-    }
-
     public static boolean isCaseSensitive(String name) {
         return name!=null && name.length() > 0 && name.charAt(0)=='"';
     }
@@ -1159,11 +1130,4 @@ public class SchemaUtil {
         }
         return false;
     }
-
-    public static boolean isNamespaceMapped(Result currentResult) {
-        Cell isNamespaceMappedCell = 
currentResult.getColumnLatestCell(TABLE_FAMILY_BYTES, 
IS_NAMESPACE_MAPPED_BYTES);
-        return isNamespaceMappedCell!=null && (boolean) 
PBoolean.INSTANCE.toObject(isNamespaceMappedCell.getValue());
-    }
-    
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java 
b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
index 24653c6..431f60b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/parse/QueryParserTest.java
@@ -26,8 +26,6 @@ import java.io.IOException;
 import java.io.StringReader;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.hadoop.hbase.util.Pair;
@@ -58,7 +56,7 @@ public class QueryParserTest {
         }
         assertEquals("Expected equality:\n" + sql + "\n" + newSQL, stmt, 
newStmt);
     }
-
+    
     private void parseQueryThatShouldFail(String sql) throws Exception {
         try {
             parseQuery(sql);
@@ -69,61 +67,6 @@ public class QueryParserTest {
     }
 
     @Test
-    public void testCreateSchema() throws Exception {
-
-        String sql0 = "create schema \"schema1\"";
-        parseQuery(sql0);
-        String sql1 = "create schema schema1";
-        parseQuery(sql1);
-        String sql2 = "create schema \"default\"";
-        parseQuery(sql2);
-        String sql3 = "create schema \"DEFAULT\"";
-        parseQuery(sql3);
-    }
-
-    @Test
-    public void testParseGrantQuery() throws Exception {
-
-        String sql0 = "GRANT 'RX' ON SYSTEM.\"SEQUENCE\" TO 'user'";
-        parseQuery(sql0);
-        String sql1 = "GRANT 'RWXCA' ON TABLE some_table0 TO 'user0'";
-        parseQuery(sql1);
-        String sql2 = "GRANT 'RWX' ON some_table1 TO 'user1'";
-        parseQuery(sql2);
-        String sql3 = "GRANT 'CA' ON SCHEMA some_schema2 TO 'user2'";
-        parseQuery(sql3);
-        String sql4 = "GRANT 'RXW' ON some_table3 TO GROUP 'group3'";
-        parseQuery(sql4);
-        String sql5 = "GRANT 'RXW' ON \"some_schema5\".\"some_table5\" TO 
GROUP 'group5'";
-        parseQuery(sql5);
-        String sql6 = "GRANT 'RWA' TO 'user6'";
-        parseQuery(sql6);
-        String sql7 = "GRANT 'A' TO GROUP 'group7'";
-        parseQuery(sql7);
-        String sql8 = "GRANT 'ARXRRRRR' TO GROUP 'group8'";
-        parseQueryThatShouldFail(sql8);
-    }
-
-    @Test
-    public void testParseRevokeQuery() throws Exception {
-
-        String sql0 = "REVOKE ON SCHEMA SYSTEM FROM 'user0'";
-        parseQuery(sql0);
-        String sql1 = "REVOKE ON SYSTEM.\"SEQUENCE\" FROM 'user1'";
-        parseQuery(sql1);
-        String sql2 = "REVOKE ON TABLE some_table2 FROM GROUP 'group2'";
-        parseQuery(sql2);
-        String sql3 = "REVOKE ON some_table3 FROM GROUP 'group2'";
-        parseQuery(sql3);
-        String sql4 = "REVOKE FROM 'user4'";
-        parseQuery(sql4);
-        String sql5 = "REVOKE FROM GROUP 'group5'";
-        parseQuery(sql5);
-        String sql6 = "REVOKE 'RRWWXAAA' FROM GROUP 'group6'";
-        parseQueryThatShouldFail(sql6);
-    }
-
-    @Test
     public void testParsePreQuery0() throws Exception {
         String sql = ((
             "select a from b\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
 
b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
index 0f12d9c..935d8cb 100644
--- 
a/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
+++ 
b/phoenix-core/src/test/java/org/apache/phoenix/query/ParallelIteratorsSplitTest.java
@@ -52,7 +52,6 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixParameterMetaData;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
-import org.apache.phoenix.optimize.Cost;
 import org.apache.phoenix.parse.FilterableStatement;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PSchema;
@@ -487,11 +486,6 @@ public class ParallelIteratorsSplitTest extends 
BaseConnectionlessQueryTest {
             public Long getEstimateInfoTimestamp() throws SQLException {
                 return null;
             }
-
-            @Override
-            public Cost getCost() {
-                return Cost.ZERO;
-            }
             
         }, null, new 
SpoolingResultIterator.SpoolingResultIteratorFactory(context.getConnection().getQueryServices()),
 context.getScan(), false, null);
         List<KeyRange> keyRanges = parallelIterators.getSplits();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
----------------------------------------------------------------------
diff --git a/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java 
b/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
index 4de9854..94ccc25 100644
--- a/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
+++ b/phoenix-pig/src/it/java/org/apache/phoenix/pig/BasePigIT.java
@@ -29,10 +29,8 @@ import java.util.Properties;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.phoenix.end2end.BaseHBaseManagedTimeIT;
 import org.apache.phoenix.end2end.Shadower;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.pig.ExecType;
@@ -64,8 +62,6 @@ public class BasePigIT extends BaseHBaseManagedTimeIT {
     public void setUp() throws Exception {
         conf = getTestClusterConfig();
         conf.set(QueryServices.EXTRA_JDBC_ARGUMENTS_ATTRIB, 
QueryServicesOptions.DEFAULT_EXTRA_JDBC_ARGUMENTS);
-        // Set CURRENT_SCN to confirm that it's ignored
-        conf.set(PhoenixRuntime.CURRENT_SCN_ATTRIB, 
Long.toString(System.currentTimeMillis()+QueryConstants.MILLIS_IN_DAY));
         pigServer = new PigServer(ExecType.LOCAL, conf);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/0b1f2274/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
----------------------------------------------------------------------
diff --git 
a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java 
b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
index e061c1c..a9f0c8f 100644
--- a/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
+++ b/phoenix-pig/src/main/java/org/apache/phoenix/pig/PhoenixHBaseStorage.java
@@ -21,9 +21,6 @@ import java.io.IOException;
 import java.sql.SQLException;
 import java.util.List;
 import java.util.Properties;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Arrays;
 
 import org.apache.commons.cli.CommandLine;
 import org.apache.commons.cli.CommandLineParser;
@@ -46,7 +43,6 @@ import org.apache.phoenix.pig.util.TableSchemaParserFunction;
 import org.apache.phoenix.pig.util.TypeUtil;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ColumnInfo;
-import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.pig.ResourceSchema;
 import org.apache.pig.ResourceSchema.ResourceFieldSchema;
 import org.apache.pig.StoreFuncInterface;
@@ -91,15 +87,14 @@ import org.slf4j.LoggerFactory;
 public class PhoenixHBaseStorage implements StoreFuncInterface {
 
     private static final Logger LOG = 
LoggerFactory.getLogger(PhoenixHBaseStorage.class);
-    private static final Set<String> PROPS_TO_IGNORE = new 
HashSet<>(Arrays.asList(PhoenixRuntime.CURRENT_SCN_ATTRIB));
-
+    
     private Configuration config;
     private RecordWriter<NullWritable, PhoenixRecordWritable> writer;
     private List<ColumnInfo> columnInfo = null;
     private String contextSignature = null;
     private ResourceSchema schema;  
     private long batchSize;
-    private final PhoenixOutputFormat outputFormat = new 
PhoenixOutputFormat<PhoenixRecordWritable>(PROPS_TO_IGNORE);
+    private final PhoenixOutputFormat outputFormat = new PhoenixOutputFormat();
     // Set of options permitted
     private final static Options validOptions = new Options();
     private final static CommandLineParser parser = new GnuParser();
@@ -233,4 +228,5 @@ public class PhoenixHBaseStorage implements 
StoreFuncInterface {
         schema = s;
         getUDFProperties().setProperty(contextSignature + SCHEMA, 
ObjectSerializer.serialize(schema));
     }
-}
+
+}
\ No newline at end of file

Reply via email to