You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ja...@apache.org on 2017/12/17 00:43:24 UTC

[02/16] phoenix git commit: PHOENIX-3837 Feature enabling to set property on an index with Alter statement

PHOENIX-3837 Feature enabling to set property on an index with Alter statement

Signed-off-by: aertoria <ca...@gmail.com>


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

Branch: refs/heads/4.x-HBase-1.2
Commit: ef174bb22ccce806dcdf5435d17319cbfa86c0d5
Parents: 5b49396
Author: aertoria <ca...@gmail.com>
Authored: Mon Nov 27 03:13:53 2017 +0000
Committer: James Taylor <jt...@salesforce.com>
Committed: Sat Dec 16 16:42:54 2017 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |   2 +-
 .../phoenix/end2end/index/IndexMetadataIT.java  |  55 ++
 phoenix-core/src/main/antlr3/PhoenixSQL.g       |   5 +-
 .../apache/phoenix/jdbc/PhoenixStatement.java   |  10 +-
 .../phoenix/parse/AddColumnStatement.java       |   2 +-
 .../phoenix/parse/AlterIndexStatement.java      |  14 +
 .../apache/phoenix/parse/ParseNodeFactory.java  |   6 +-
 .../phoenix/query/ConnectionQueryServices.java  |   2 +
 .../query/ConnectionQueryServicesImpl.java      |  20 +
 .../query/ConnectionlessQueryServicesImpl.java  |   7 +
 .../query/DelegateConnectionQueryServices.java  |   8 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 566 +++++++++++++------
 12 files changed, 520 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 5265b09..17f08c4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -1080,7 +1080,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 			} catch (SQLException e) {
 				assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED.getErrorCode(), e.getErrorCode());
 			}
-			// altering a table to be transactional  should fail if transactions are disabled
+			// altering a table to be transactional should fail if transactions are disabled
 			conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
 			try {
 				conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " SET TRANSACTIONAL=true");

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
index 0ce36dd..986c317 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexMetadataIT.java
@@ -674,4 +674,59 @@ public class IndexMetadataIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
+
+
+
+    @Test
+    public void testIndexAlterPhoenixProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String testTable = generateUniqueName();
+
+
+        String ddl = "create table " + testTable  + " (k varchar primary key, v1 varchar)";
+        Statement stmt = conn.createStatement();
+        stmt.execute(ddl);
+        String indexName = "IDX_" + generateUniqueName();
+
+        ddl = "CREATE INDEX " + indexName + " ON " + testTable  + " (v1) ";
+        stmt.execute(ddl);
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET GUIDE_POSTS_WIDTH = 10");
+
+        ResultSet rs = conn.createStatement().executeQuery(
+                "select GUIDE_POSTS_WIDTH from SYSTEM.\"CATALOG\" where TABLE_NAME='" + indexName + "'");assertTrue(rs.next());
+        assertEquals(10,rs.getInt(1));
+
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET GUIDE_POSTS_WIDTH = 20");
+        rs = conn.createStatement().executeQuery(
+                "select GUIDE_POSTS_WIDTH from SYSTEM.\"CATALOG\" where TABLE_NAME='" + indexName + "'");assertTrue(rs.next());
+        assertEquals(20,rs.getInt(1));
+    }
+
+
+    @Test
+    public void testIndexAlterHBaseProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String testTable = generateUniqueName();
+
+        String ddl = "create table " + testTable  + " (k varchar primary key, v1 varchar)";
+        Statement stmt = conn.createStatement();
+        stmt.execute(ddl);
+        String indexName = "IDX_" + generateUniqueName();
+
+        ddl = "CREATE INDEX " + indexName + " ON " + testTable  + " (v1) ";
+        stmt.execute(ddl);
+
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET DISABLE_WAL=false");
+        asssertIsWALDisabled(conn,indexName,false);
+        conn.createStatement().execute("ALTER INDEX "+indexName+" ON " + testTable +" ACTIVE SET DISABLE_WAL=true");
+        asssertIsWALDisabled(conn,indexName,true);
+    }
+
+    private static void asssertIsWALDisabled(Connection conn, String fullTableName, boolean expectedValue) throws SQLException {
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        assertEquals(expectedValue, pconn.getTable(new PTableKey(pconn.getTenantId(), fullTableName)).isWALDisabled());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/antlr3/PhoenixSQL.g
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/antlr3/PhoenixSQL.g b/phoenix-core/src/main/antlr3/PhoenixSQL.g
index 8c9c135..ccf654b 100644
--- a/phoenix-core/src/main/antlr3/PhoenixSQL.g
+++ b/phoenix-core/src/main/antlr3/PhoenixSQL.g
@@ -605,8 +605,9 @@ drop_index_node returns [DropIndexStatement ret]
 
 // Parse a alter index statement
 alter_index_node returns [AlterIndexStatement ret]
-    : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name s=(USABLE | UNUSABLE | REBUILD | DISABLE | ACTIVE) (async=ASYNC)?
-      {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText())), async!=null); }
+    : ALTER INDEX (IF ex=EXISTS)? i=index_name ON t=from_table_name
+      ((s=(USABLE | UNUSABLE | REBUILD | DISABLE | ACTIVE)) (async=ASYNC)? ((SET?)p=fam_properties)?)
+      {ret = factory.alterIndex(factory.namedTable(null, TableName.create(t.getSchemaName(), i.getName())), t.getTableName(), ex!=null, PIndexState.valueOf(SchemaUtil.normalizeIdentifier(s.getText())), async!=null, p); }
     ;
 
 // Parse a trace statement.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
index 26b4415..c699088 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixStatement.java
@@ -1216,8 +1216,8 @@ public class PhoenixStatement implements Statement, SQLCloseable {
 
     private static class ExecutableAlterIndexStatement extends AlterIndexStatement implements CompilableStatement {
 
-        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
-            super(indexTableNode, dataTableName, ifExists, state, async);
+        public ExecutableAlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async, ListMultimap<String,Pair<String,Object>> props) {
+            super(indexTableNode, dataTableName, ifExists, state, async, props);
         }
 
         @SuppressWarnings("unchecked")
@@ -1563,10 +1563,10 @@ public class PhoenixStatement implements Statement, SQLCloseable {
         public DropIndexStatement dropIndex(NamedNode indexName, TableName tableName, boolean ifExists) {
             return new ExecutableDropIndexStatement(indexName, tableName, ifExists);
         }
-        
+
         @Override
-        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
-            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
+        public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async, ListMultimap<String,Pair<String,Object>> props) {
+            return new ExecutableAlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async, props);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
index 1890d31..678e560 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AddColumnStatement.java
@@ -48,4 +48,4 @@ public class AddColumnStatement extends AlterTableStatement {
     public ListMultimap<String,Pair<String,Object>> getProps() {
         return props;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
index 11328c2..de04505 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/AlterIndexStatement.java
@@ -17,20 +17,31 @@
  */
 package org.apache.phoenix.parse;
 
+import com.google.common.collect.ImmutableListMultimap;
+import com.google.common.collect.ListMultimap;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTableType;
 
 public class AlterIndexStatement extends SingleTableStatement {
     private final String dataTableName;
     private final boolean ifExists;
     private final PIndexState indexState;
     private boolean async;
+    private ListMultimap<String,Pair<String,Object>> props;
+    private static final PTableType tableType=PTableType.INDEX;
 
     public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState, boolean async) {
+        this(indexTableNode,dataTableName,ifExists,indexState,async,null);
+    }
+
+    public AlterIndexStatement(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState indexState, boolean async, ListMultimap<String,Pair<String,Object>> props) {
         super(indexTableNode,0);
         this.dataTableName = dataTableName;
         this.ifExists = ifExists;
         this.indexState = indexState;
         this.async = async;
+        this.props= props==null ? ImmutableListMultimap.<String,Pair<String,Object>>of() : props;
     }
 
     public String getTableName() {
@@ -54,4 +65,7 @@ public class AlterIndexStatement extends SingleTableStatement {
         return async;
     }
 
+    public ListMultimap<String,Pair<String,Object>> getProps() { return props; }
+
+    public PTableType getTableType(){ return tableType; }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
index 32c3d8d..9be59f3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/parse/ParseNodeFactory.java
@@ -373,10 +373,10 @@ public class ParseNodeFactory {
         return new DropIndexStatement(indexName, tableName, ifExists);
     }
 
-    public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async) {
-        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async);
+    public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state, boolean async, ListMultimap<String,Pair<String,Object>> props) {
+        return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, async, props);
     }
-    
+
     public AlterIndexStatement alterIndex(NamedTableNode indexTableNode, String dataTableName, boolean ifExists, PIndexState state) {
         return new AlterIndexStatement(indexTableNode, dataTableName, ifExists, state, false);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index 45ab5fa..90f8089 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -86,6 +86,8 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
     public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException;
     public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException;
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName,  Map<String, List<Pair<String,Object>>> stmtProperties,  PTable table) throws SQLException;
+
     public MutationState updateData(MutationPlan plan) throws SQLException;
 
     public void init(String url, Properties props) throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/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 08aadfb..072bf28 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
@@ -3601,6 +3601,26 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
+    public MetaDataMutationResult updateIndexState(final List<Mutation> tableMetaData, String parentTableName, Map<String, List<Pair<String,Object>>> stmtProperties,  PTable table) throws SQLException {
+        if(stmtProperties==null) return updateIndexState(tableMetaData,parentTableName);
+
+        Map<String, Object> tableProps = new HashMap<String, Object>();
+        Pair<HTableDescriptor,HTableDescriptor> tableDescriptorPair = separateAndValidateProperties(table, stmtProperties, new HashSet<String>(), tableProps);
+        HTableDescriptor tableDescriptor = tableDescriptorPair.getSecond();
+        HTableDescriptor origTableDescriptor = tableDescriptorPair.getFirst();
+        Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
+        Set<HTableDescriptor> origTableDescriptors = Collections.emptySet();
+        if (tableDescriptor != null) {
+            tableDescriptors = Sets.newHashSetWithExpectedSize(3 + table.getIndexes().size());
+            origTableDescriptors = Sets.newHashSetWithExpectedSize(3 + table.getIndexes().size());
+            tableDescriptors.add(tableDescriptor);
+            origTableDescriptors.add(origTableDescriptor);
+        }
+        sendHBaseMetaData(tableDescriptors, true);
+        return updateIndexState(tableMetaData,parentTableName);
+    }
+
+    @Override
     public long createSequence(String tenantId, String schemaName, String sequenceName,
             long startWith, long incrementBy, long cacheSize, long minValue, long maxValue,
             boolean cycle, long timestamp) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index f15e0b1..3154f86 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -383,6 +383,13 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
+    public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata,
+            String parentTableName, Map<String, List<Pair<String, Object>>> stmtProperties,
+            PTable table) throws SQLException {
+        return updateIndexState(tableMetadata,parentTableName);
+    }
+
+    @Override
     public HTableDescriptor getTableDescriptor(byte[] tableName) throws SQLException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 6c464eb..05d1af6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -135,7 +135,13 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException {
         return getDelegate().updateIndexState(tableMetadata, parentTableName);
     }
-    
+
+    @Override public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata,
+            String parentTableName, Map<String, List<Pair<String, Object>>> stmtProperties,
+            PTable table) throws SQLException {
+        return getDelegate().updateIndexState(tableMetadata, parentTableName, stmtProperties,table);
+    }
+
     @Override
     public void init(String url, Properties props) throws SQLException {
         getDelegate().init(url, props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ef174bb2/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 1f76e90..fc2e288 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
@@ -3007,6 +3007,21 @@ 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);
     }
@@ -3138,18 +3153,7 @@ 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
@@ -3185,48 +3189,14 @@ 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;
-            boolean nonTxToTx = false;
+            MetaProperties metaProperties = new MetaProperties();
             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();
@@ -3253,108 +3223,11 @@ 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<>();
@@ -3471,7 +3344,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(isImmutableRows) && !table.getIndexes().isEmpty()) {
+                    if (Boolean.FALSE.equals(metaPropertiesEvaluated.getIsImmutableRows()) && !table.getIndexes().isEmpty()) {
                         int hbaseVersion = connection.getQueryServices().getLowestClusterHBaseVersion();
                         if (hbaseVersion < PhoenixDatabaseMetaData.MUTABLE_SI_VERSION_THRESHOLD) {
                             throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES)
@@ -3482,22 +3355,22 @@ public class MetaDataClient {
                             .setSchemaName(schemaName).setTableName(tableName).build().buildException();
                         }
                     }
-                    if (Boolean.TRUE.equals(multiTenant)) {
-                        throwIfInsufficientColumns(schemaName, tableName, table.getPKColumns(), table.getBucketNum()!=null, multiTenant);
+                    if (Boolean.TRUE.equals(metaPropertiesEvaluated.getMultiTenant())) {
+                        throwIfInsufficientColumns(schemaName, tableName, table.getPKColumns(), table.getBucketNum()!=null, metaPropertiesEvaluated.getMultiTenant());
                     }
                 }
 
-                if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || nonTxToTx)) {
+                if (!table.getIndexes().isEmpty() && (numPkColumnsAdded>0 || metaProperties.getNonTxToTx())) {
                     for (PTable index : table.getIndexes()) {
-                        incrementTableSeqNum(index, index.getType(), numPkColumnsAdded, nonTxToTx ? Boolean.TRUE : null, updateCacheFrequency);
+                        incrementTableSeqNum(index, index.getType(), numPkColumnsAdded, metaProperties.getNonTxToTx() ? Boolean.TRUE : null, metaPropertiesEvaluated.getUpdateCacheFrequency());
                     }
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
                 
                 if (changingPhoenixTableProperty || columnDefs.size() > 0) {
-                    incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows,
-                            disableWAL, multiTenant, storeNulls, guidePostWidth, appendOnlySchema, immutableStorageScheme, useStatsForParallelization);
+                    incrementTableSeqNum(table, tableType, columnDefs.size(), metaPropertiesEvaluated);
+
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -3567,10 +3440,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 && !nonTxToTx)) {
+                    if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && ! metaProperties.getNonTxToTx())) {
                         connection.addTable(result.getTable(), resolvedTimeStamp);
                         table = result.getTable();
-                    } else if (updateCacheFrequency != null) {
+                    } else if (metaPropertiesEvaluated.getUpdateCacheFrequency() != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
                         connection.removeTable(tenantId, fullTableName, null, resolvedTimeStamp);
@@ -3578,7 +3451,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(multiTenant)
+                            && Boolean.FALSE.equals(metaPropertiesEvaluated.getMultiTenant())
                             && MetaDataUtil.hasViewIndexTable(connection, table.getPhysicalName())) {
                         connection.setAutoCommit(true);
                         MetaDataUtil.deleteViewIndexSequences(connection, table.getPhysicalName(), table.isNamespaceMapped());
@@ -3940,9 +3813,19 @@ 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)
@@ -3980,7 +3863,15 @@ public class MetaDataClient {
             List<Mutation> tableMetadata = connection.getMutationState().toMutations(timeStamp).next().getSecond();
             connection.rollback();
 
-            MetaDataMutationResult result = connection.getQueryServices().updateIndexState(tableMetadata, dataTableName);
+
+            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);
+
             MutationCode code = result.getMutationCode();
             if (code == MutationCode.TABLE_NOT_FOUND) {
                 throw new TableNotFoundException(schemaName,indexName);
@@ -4306,4 +4197,351 @@ public class MetaDataClient {
             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;
+        }
+    }
 }