You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2017/01/25 18:50:29 UTC

[2/2] phoenix git commit: PHOENIX-3519 Specify QualifierEncodingScheme in CREATE TABLE statement

PHOENIX-3519 Specify QualifierEncodingScheme in CREATE TABLE statement


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

Branch: refs/heads/encodecolumns2
Commit: 61d9035cd09bd1fff834dccd03a454df803e521a
Parents: c004c6e
Author: Thomas D'Silva <td...@salesforce.com>
Authored: Mon Jan 9 19:22:19 2017 -0800
Committer: Thomas D'Silva <td...@salesforce.com>
Committed: Wed Jan 25 09:18:48 2017 -0800

----------------------------------------------------------------------
 .../AlterMultiTenantTableWithViewsIT.java       |  25 +-
 .../apache/phoenix/end2end/AlterTableIT.java    | 237 ++++++++-------
 .../phoenix/end2end/AlterTableWithViewsIT.java  | 210 +++++++------
 .../end2end/ColumnEncodedBytesPropIT.java       |  95 ++++++
 .../phoenix/end2end/StatsCollectorIT.java       | 115 ++++++--
 .../apache/phoenix/end2end/StoreNullsIT.java    | 291 ++++++++++---------
 .../phoenix/end2end/StoreNullsPropIT.java       |  51 ++++
 ...SysTableNamespaceMappedStatsCollectorIT.java |   4 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |   1 -
 .../phoenix/end2end/index/MutableIndexIT.java   |   4 +
 .../phoenix/exception/SQLExceptionCode.java     |   2 +
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   3 +
 .../apache/phoenix/query/QueryConstants.java    |   2 +-
 .../org/apache/phoenix/query/QueryServices.java |   2 +
 .../phoenix/query/QueryServicesOptions.java     |   2 +
 .../apache/phoenix/schema/MetaDataClient.java   |  47 +--
 .../apache/phoenix/schema/TableProperty.java    |  21 ++
 .../phoenix/compile/QueryOptimizerTest.java     |   4 +-
 .../phoenix/execute/MutationStateTest.java      |   4 +-
 .../java/org/apache/phoenix/query/BaseTest.java |  19 +-
 .../phoenix/query/ConnectionlessTest.java       |   4 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |  46 ++-
 22 files changed, 769 insertions(+), 420 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/61d9035c/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index 8275f3f..adadca7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.junit.Assert.assertEquals;
@@ -477,14 +476,14 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             // For a diverged view, only base table's pk column will be added and that too at the end.
             assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
             
-            // Adding existing column VIEW_COL2 to the base table isn't allowed.
+            // Add existing column VIEW_COL2 to the base table
             alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
-            try {
-                conn.createStatement().execute(alterBaseTable);
-                fail();
-            } catch (SQLException e) {
-                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-            }
+            conn.createStatement().execute(alterBaseTable);
+            
+            // For the non-diverged view, adding the column VIEW_COL2 will end up changing its ordinal position in the view.
+            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 2, 9, 8, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL2", "VIEW_COL1");
+            // For the diverged view, adding the column VIEW_COL2 will not change its ordinal position in the view. It also won't change the base column count or the sequence number
+            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
         }
     }
     
@@ -501,13 +500,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the both columns were added to view1
@@ -531,13 +530,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the dropped columns aren't visible
@@ -570,7 +569,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
                 tenant1Conn.createStatement().execute(view1DDL);
                 // This should not modify the base table
-                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR, NEWCOL5 VARCHAR";
+                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR";
                 tenant1Conn.createStatement().execute(alterView);
                 HTableDescriptor tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
                 assertEquals(tableDesc1, tableDesc2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/61d9035c/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 c2fb031..9799b08 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
@@ -44,6 +44,8 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -73,6 +75,9 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  *
@@ -85,14 +90,28 @@ import org.junit.Test;
  * or at the end of test class.
  *
  */
+@RunWith(Parameterized.class)
 public class AlterTableIT extends ParallelStatsDisabledIT {
     private String schemaName;
     private String dataTableName;
     private String indexTableName;
     private String localIndexTableName;
+    private String viewName;
     private String dataTableFullName;
     private String indexTableFullName;
     private String localIndexTableFullName;
+    private String tableDDLOptions;
+    private final boolean columnEncoded;
+    
+    public AlterTableIT(boolean columnEncoded) {
+        this.columnEncoded = columnEncoded;
+        this.tableDDLOptions = columnEncoded ? "COLUMN_ENCODED_BYTES=4" : "";
+    }
+    
+    @Parameters(name="AlterTableIT_columnEncoded={0}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean> data() {
+        return Arrays.asList( false, true);
+    }
 
     @Before
     public void setupTableNames() throws Exception {
@@ -103,6 +122,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
         indexTableFullName = SchemaUtil.getTableName(schemaName, indexTableName);
         localIndexTableFullName = SchemaUtil.getTableName(schemaName, localIndexTableName);
+        viewName = generateUniqueName();
     }
 
     @Test
@@ -114,7 +134,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         try {
             String ddl = "CREATE TABLE  " + dataTableFullName +
                     "  (a_string varchar not null, a_binary varbinary not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n";
+                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary)) " + tableDDLOptions;
             createTestTable(getUrl(), ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " ADD b_string VARCHAR NULL PRIMARY KEY";
@@ -155,7 +175,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         try {
             String ddl = "CREATE TABLE " + dataTableFullName +
                     "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             String dml = "UPSERT INTO " + dataTableFullName + " VALUES(?)";
@@ -222,7 +242,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String ddl = "CREATE TABLE  " + dataTableFullName +
                 "  (a_string varchar not null, col1 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " ADD CF.col2 integer CF.IN_MEMORY=true");
@@ -255,6 +275,19 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         helpTestDropIndexedColumn(false);
     }
     
+    private String generateDDLOptions(String options) {
+        StringBuilder sb = new StringBuilder();
+        if (!options.isEmpty()) {
+            sb.append(options);
+        }
+        if (!tableDDLOptions.isEmpty()) {
+            if (sb.length()!=0)
+                sb.append(",");
+            sb.append(tableDDLOptions);
+        }
+        return sb.toString();
+    }
+    
     private void helpTestDropIndexedColumn(boolean immutable) throws Exception {
         String query;
         ResultSet rs;
@@ -267,7 +300,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         // make sure that the tables are empty, but reachable
         conn.createStatement().execute(
           "CREATE TABLE " + dataTableFullName
-              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + (immutable ? "IMMUTABLE_ROWS = true" : ""));
+              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
+              + generateDDLOptions((immutable ? "IMMUTABLE_ROWS = true" : "")));
         query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
@@ -327,7 +361,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         // make sure that the tables are empty, but reachable
         conn.createStatement().execute(
           "CREATE TABLE " + dataTableFullName
-              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR)");
+              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR) " +  tableDDLOptions);
         String dataTableQuery = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(dataTableQuery);
         assertFalse(rs.next());
@@ -442,7 +476,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         // make sure that the tables are empty, but reachable
         conn.createStatement().execute(
           "CREATE TABLE " + dataTableFullName
-              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions);
         query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
@@ -538,7 +572,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             		"        B.M_TEXT VARCHAR\n" +
             		"        CONSTRAINT ROWKEY PRIMARY KEY\n" +
             		"(SENDER_ID,RECIPIENT_ID,M_TIMESTAMP DESC,ROW_ID))\n" +
-            		"SALT_BUCKETS=4";
+            		generateDDLOptions("SALT_BUCKETS=4");
             conn.createStatement().execute(ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " SET IMMUTABLE_ROWS=true";
@@ -572,7 +606,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     "        B.M_TEXT VARCHAR\n" +
                     "        CONSTRAINT ROWKEY PRIMARY KEY\n" +
                     "(SENDER_ID,RECIPIENT_ID,M_TIMESTAMP DESC,ROW_ID))\n" +
-                    "SALT_BUCKETS=4";
+                    generateDDLOptions("SALT_BUCKETS=4");
             conn.createStatement().execute(ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " DROP COLUMN B.JSON";
@@ -595,7 +629,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         try {
             String ddl = "CREATE TABLE " + dataTableFullName +
                     "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             String dml = "UPSERT INTO " + dataTableFullName + " VALUES(?)";
@@ -702,7 +736,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         conn.setAutoCommit(false);
         try {
             String ddl = "CREATE TABLE " + dataTableFullName + " " + "  (a_string varchar not null, col1 integer, cf1.col2 integer"
-                    + "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " DROP COLUMN col1";
@@ -725,7 +759,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             conn.setAutoCommit(false);
             try {
                 String ddl = "CREATE TABLE " + dataTableFullName + " " + "  (a_string varchar not null, col1 integer, cf1.col2 integer"
-                        + "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                        + "  CONSTRAINT pk PRIMARY KEY (a_string)) " +  tableDDLOptions;
                 stmt = conn.prepareStatement(ddl);
                 stmt.execute();
             } finally {
@@ -760,7 +794,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true, disable_wal=true ");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) "
+                                    + generateDDLOptions("immutable_rows=true, disable_wal=true"));
 
             Connection conn2 = DriverManager.getConnection(getUrl(), props);
             String query = "SELECT * FROM " + dataTableFullName;
@@ -792,7 +827,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string))" 
+                                    + generateDDLOptions("immutable_rows=true"));
 
             Connection conn2 = DriverManager.getConnection(getUrl(), props);
             String query = "SELECT * FROM " + dataTableFullName;
@@ -825,7 +861,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string))");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions);
 
             Connection conn2 = DriverManager.getConnection(getUrl(), props);
             String query = "SELECT * FROM " + dataTableFullName;
@@ -863,7 +899,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true , SALT_BUCKETS=3 ");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) " 
+                                    + generateDDLOptions("immutable_rows=true , SALT_BUCKETS=3 "));
 
             String query = "SELECT * FROM " + dataTableFullName;
             ResultSet rs = conn.createStatement().executeQuery(query);
@@ -952,7 +989,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
         // here we insert into the orig schema with one column
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
-        conn1.createStatement().execute("create table " + dataTableFullName + "(id VARCHAR PRIMARY KEY, field1 BIGINT)");
+        conn1.createStatement().execute("create table " + dataTableFullName + "(id VARCHAR PRIMARY KEY, field1 BIGINT) " + tableDDLOptions);
         PreparedStatement stmtInsert1 = conn1.prepareStatement("upsert into " + dataTableFullName + " (id, field1) values ( ?, ?)");
         stmtInsert1.setString(1, "key1");
         stmtInsert1.setLong(2, 1L);
@@ -984,7 +1021,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2))";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) "  + tableDDLOptions;
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " ADD STRING VARCHAR, STRING_DATA_TYPES VARCHAR";
@@ -1001,7 +1038,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl = "CREATE TABLE " + dataTableFullName + " (\n"
                 +"ID VARCHAR(15) PRIMARY KEY,\n"
-                +"COL1 BIGINT)";
+                +"COL1 BIGINT) " + tableDDLOptions;
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         conn1.createStatement().execute("CREATE INDEX " + indexTableName + " ON " + dataTableFullName + "(COL1)");
@@ -1058,7 +1095,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 + "COL1 BIGINT,"
                 + "COL2 BIGINT,"
                 + "COL3 BIGINT,"
-                + "COL4 BIGINT)";
+                + "COL4 BIGINT) " + tableDDLOptions;
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         conn1.createStatement().execute("CREATE INDEX " + indexTableName + " ON " + dataTableFullName + "(COL1) INCLUDE (COL2,COL3,COL4)");
@@ -1114,7 +1151,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET REPLICATION_SCOPE=1";
@@ -1137,7 +1174,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET COMPACTION_ENABLED=FALSE";
@@ -1159,7 +1196,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET COMPACTION_ENABLED = FALSE, REPLICATION_SCOPE = 1";
@@ -1183,7 +1220,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CF1.CREATION_TIME BIGINT,\n"
                 +"CF2.LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) IMMUTABLE_ROWS=true";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("IMMUTABLE_ROWS=true");
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         assertImmutableRows(conn, dataTableFullName, true);
@@ -1225,7 +1262,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.COMPACTION_ENABLED = FALSE";
@@ -1246,7 +1283,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.DISABLE_WAL = TRUE";
@@ -1267,7 +1304,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"CF.LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.TTL = 86400";
@@ -1288,7 +1325,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.REPLICATION_SCOPE = 1";
@@ -1309,7 +1346,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions(" SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET DEFAULT_COLUMN_FAMILY = 'A'";
@@ -1330,19 +1367,19 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v AS SELECT * FROM " + dataTableFullName + " WHERE CREATION_TIME = 1";
+        ddl = "CREATE VIEW " + viewName + "  AS SELECT * FROM " + dataTableFullName + " WHERE CREATION_TIME = 1";
         conn1.createStatement().execute(ddl);
-        ddl = "ALTER VIEW v SET REPLICATION_SCOPE = 1";
+        ddl = "ALTER VIEW " + viewName + " SET REPLICATION_SCOPE = 1";
         try {
             conn1.createStatement().execute(ddl);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.VIEW_WITH_PROPERTIES.getErrorCode(), e.getErrorCode());
         }
-        ddl = "ALTER VIEW v SET COMPACTION_ENABLED = FALSE";
+        ddl = "ALTER VIEW " + viewName + " SET COMPACTION_ENABLED = FALSE";
         try {
             conn1.createStatement().execute(ddl);
             fail();
@@ -1360,7 +1397,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         String viewFullName = SchemaUtil.getTableName(schemaName, generateUniqueName());
@@ -1395,7 +1432,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"CF.LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) IMMUTABLE_ROWS=true, DEFAULT_COLUMN_FAMILY = 'XYZ'";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("IMMUTABLE_ROWS=true, DEFAULT_COLUMN_FAMILY = 'XYZ'");
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         assertImmutableRows(conn, dataTableFullName, true);
@@ -1429,7 +1466,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName
                 +
                 "  (a_string varchar not null, col1 integer, CF.col2 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute(
@@ -1455,7 +1492,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName + " "
                 +
                 "  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement()
@@ -1489,7 +1526,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName + " "
                 +
                 "  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ'");
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement()
@@ -1523,7 +1560,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName + " "
                 +
     			"  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-    			"  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+    			"  CONSTRAINT pk PRIMARY KEY (a_string)) "+ generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ'");
     	try {
     		conn.createStatement().execute(ddl);
     		try {
@@ -1546,7 +1583,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName
                 +
                 "  (a_string varchar not null, col1 integer, CF1.col2 integer, CF2.col3 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ' ");
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement()
@@ -1583,7 +1620,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName
                 +
     			"  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-    			"  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+    			"  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ'");
     	try {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute(
@@ -1613,7 +1650,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     	conn.setAutoCommit(false);
         String ddl = "CREATE TABLE " + dataTableFullName +
     			"  (a_string varchar not null, col1 integer" +
-    			"  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+    			"  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
     	try {
     		conn.createStatement().execute(ddl);
             conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " ADD col2 integer IN_MEMORY=true");
@@ -1641,7 +1678,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                             "CREATE TABLE "
                                     + dataTableFullName
     						+ "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true , SALT_BUCKETS=3 ");
+    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("immutable_rows=true , SALT_BUCKETS=3 "));
 
             String ddl = "Alter table " + dataTableFullName + " add cf3.col5 integer, cf4.col6 integer in_memory=true";
     		conn.createStatement().execute(ddl);
@@ -1679,7 +1716,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                             "CREATE TABLE "
                                     + dataTableFullName
     						+ "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true , SALT_BUCKETS=3 ");
+    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("immutable_rows=true , SALT_BUCKETS=3 "));
 
             String ddl = "Alter table " + dataTableFullName + " add cf1.col5 integer in_memory=true";
     		conn.createStatement().execute(ddl);
@@ -1709,7 +1746,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
     	try {
             String ddl = "CREATE TABLE " + dataTableFullName
-                    + " (pk char(2) not null primary key, col1 integer, b.col1 integer) SPLIT ON ('EA','EZ')";
+                    + " (pk char(2) not null primary key, col1 integer, b.col1 integer) " + tableDDLOptions + " SPLIT ON ('EA','EZ') ";
     		conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " add b.col2 varchar ttl=30";
     		conn.createStatement().execute(ddl);
@@ -1729,7 +1766,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
     	try {
             String ddl = "CREATE TABLE " + dataTableFullName
-                    + " (pk char(2) not null primary key) TTL=100 SPLIT ON ('EA','EZ')";
+                    + " (pk char(2) not null primary key) " + generateDDLOptions("TTL=100") + " SPLIT ON ('EA','EZ')";
     		conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " add col1 varchar ttl=30";
     		conn.createStatement().execute(ddl);
@@ -1759,7 +1796,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
                 HTableDescriptor tableDesc = admin.getTableDescriptor(Bytes.toBytes(dataTableFullName));
@@ -1794,7 +1831,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " SET IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1822,7 +1859,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " SET IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1850,7 +1887,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD COL3 INTEGER IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1878,7 +1915,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD NEWCF.COL3 INTEGER IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1908,7 +1945,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD NEWCF.COL3 INTEGER IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1972,7 +2009,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     				+ " col1 integer NOT NULL,"
     				+ " col2 bigint NOT NULL,"
     				+ " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-    				+ " )";
+    				+ " ) " +  tableDDLOptions;
     		conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD NEWCF.COL3 INTEGER NEWCF.UNKNOWN_PROP='ABC'";
     		try {
@@ -1998,7 +2035,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
 
         Statement stmt = conn.createStatement();
-        stmt.execute("CREATE TABLE " + dataTableFullName + " (id SMALLINT PRIMARY KEY, name VARCHAR)");
+        stmt.execute("CREATE TABLE " + dataTableFullName + " (id SMALLINT PRIMARY KEY, name VARCHAR) "+tableDDLOptions);
 
         ResultSet rs = stmt.executeQuery("SELECT STORE_NULLS FROM SYSTEM.CATALOG " +
  "WHERE table_name = '"
@@ -2030,7 +2067,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     + " k2 integer NOT NULL,"
                     + " col1 bigint,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (k1, k2)"
-                    + " )";
+                    + " ) "+tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             // set HTableProperty when adding a pk column should fail
@@ -2104,7 +2141,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     + " col1 integer NOT NULL,"
                     + " col2 bigint NOT NULL,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-                    + " )";
+                    + " ) "+tableDDLOptions;
             conn.createStatement().execute(ddl);
             asssertIsWALDisabled(conn, dataTableFullName, false);
             
@@ -2141,7 +2178,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     @Test
     public void testDeclaringColumnAsRowTimestamp() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) ");
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
             PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class); 
             PTable table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName));
             // Assert that the column shows up as row time stamp in the cache.
@@ -2199,7 +2236,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     @Test
     public void testAddingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 VARCHAR NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2)) ");
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 VARCHAR NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2)) " + tableDDLOptions);
             // adding a new pk column that is also row_timestamp is not allowed
             try {
                 conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " ADD PK3 DATE PRIMARY KEY ROW_TIMESTAMP");
@@ -2217,7 +2254,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 		try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 			// creating a transactional table should fail if transactions are disabled
 			try {
-				conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR) TRANSACTIONAL=true");
+				conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR) " + generateDDLOptions("TRANSACTIONAL=true"));
 				fail();
 			} catch (SQLException e) {
 				assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED.getErrorCode(), e.getErrorCode());
@@ -2239,7 +2276,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             String tableName = generateUniqueName();
             conn.createStatement().execute(
                     "CREATE TABLE " + tableName
-                    + " (k1 VARCHAR NOT NULL, k2 VARCHAR, CONSTRAINT PK PRIMARY KEY(K1,K2)) ");
+                    + " (k1 VARCHAR NOT NULL, k2 VARCHAR, CONSTRAINT PK PRIMARY KEY(K1,K2)) " + tableDDLOptions);
             try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
                 admin.disableTable(Bytes.toBytes(tableName));
             }
@@ -2256,7 +2293,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     }
 
 	@Test
-	public void testMetadataForImmutableTableWithEncodedColumns() throws Exception {
+	public void testMetadataForImmutableTable() throws Exception {
 	    String schemaName = "XYZ";
 	    String baseTableName = generateUniqueName();
 	    String viewName = generateUniqueName();
@@ -2270,17 +2307,17 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 	                + " COL2 bigint NOT NULL,"
 	                + " KV1 VARCHAR"
 	                + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
-	                + " )  IMMUTABLE_ROWS = true");
+	                + " ) " + generateDDLOptions("IMMUTABLE_ROWS = true"));
 	        PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
 	        long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
 
 	        // assert that the client side cache is updated.
 	        EncodedCQCounter cqCounter = baseTable.getEncodedCQCounter();
-	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
+	        assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1) : null, cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
 	        
 	        
 	        // assert that the server side metadata is updated correctly.
-	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
 	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "KV1", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
 	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber);
 
@@ -2291,22 +2328,22 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 	        PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
 
 	        // verify that the client side cache is updated. Base table's cq counters should be updated.
-	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2), baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
-	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), baseTable.getEncodedCQCounter().getNextQualifier("A"));
+	        assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2) : null, baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1) : null, baseTable.getEncodedCQCounter().getNextQualifier("A"));
 	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
 	        
 	        // assert that the server side metadata for the base table and the view is also updated correctly.
-	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2, true);
-	        assertEncodedCQCounter("A", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+	        assertEncodedCQCounter("A", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
 	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
 	        assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
-	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 1);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + (columnEncoded ? 1 : 0));
 	        assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM);
 	    }
 	}
 	
 	@Test
-	public void testMetadataForMutableTableWithEncodedColumns() throws Exception {
+	public void testMetadataForMutableTable() throws Exception {
 	    String schemaName = "XYZ";
 	    String baseTableName = generateUniqueName();
 	    String viewName = generateUniqueName();
@@ -2320,17 +2357,17 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 	                + " COL2 bigint NOT NULL,"
 	                + " KV1 VARCHAR"
 	                + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
-	                + " )");
+	                + " ) " + tableDDLOptions);
 	        PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
 	        long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
 
 	        // assert that the client side cache is updated.
 	        EncodedCQCounter cqCounter = baseTable.getEncodedCQCounter();
-	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
+	        assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1) : null, cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
 
 
 	        // assert that the server side metadata is updated correctly.
-	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
 	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "KV1", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
 	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber);
 
@@ -2341,21 +2378,21 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 	        PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
 
 	        // verify that the client side cache is updated. Base table's cq counters should be updated.
-	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3), baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        assertEquals(columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3) : null, baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
 	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
 
 	        // assert that the server side metadata for the base table and the view is also updated correctly.
-	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3, true);
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
 	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
 	        assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
-	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 1);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + (columnEncoded ? 1 : 0));
 	        assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM);
 	    }
 	}
 	
 	@Test
-    public void testAddingColumnsToTablesAndViewsWithEncodedColumns() throws Exception {
-        String schemaName = "XYZ";
+    public void testAddingColumnsToTablesAndViews() throws Exception {
+        String schemaName = generateUniqueName();
         String baseTableName = generateUniqueName();
         String viewName = generateUniqueName();
         String fullTableName = schemaName + "." + baseTableName;
@@ -2370,7 +2407,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     + " COL1 integer NOT NULL,"
                     + " COL2 bigint NOT NULL,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
-                    + " )");
+                    + " ) " + tableDDLOptions);
             PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
             long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
 
@@ -2381,30 +2418,30 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             // assert that the client side cache is updated.
             baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
             EncodedCQCounter encodedCqCounter = baseTable.getEncodedCQCounter();
-            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3), encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            assertEquals( columnEncoded ?(Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3) : null, encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
             
             // assert that the server side metadata is updated correctly.
-            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3, true);
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
             
             // assert that the server side metadata for columns is updated correctly.
             assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL4", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
             assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL5", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
             assertEncodedCQValue("B", "COL6", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
-            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 1);
+            long baseTableSeqNumBeforeAddingChildCols = initBaseTableSeqNumber + 1;
+            assertSequenceNumber(schemaName, baseTableName, baseTableSeqNumBeforeAddingChildCols);
 
             // Create a view
             String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
             conn.createStatement().execute(viewDDL);
             
             // assert that the server side metadata is updated correctly.
-            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5, true);
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5);
             
             // assert that the server side metadata for columns is updated correctly.
             assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
             assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 4);
-            // Creating a view that adds its own columns should increment the base table's sequence number too.
-            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 2);
-            
+            // for encoded columns creating a view that adds its own columns should increment the base table's sequence number too.
+            assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 2 : baseTableSeqNumBeforeAddingChildCols );
 
             // Add column to the view
             viewDDL = "ALTER VIEW " + fullViewName + " ADD VIEW_COL3 DECIMAL(10, 2), A.VIEW_COL4 VARCHAR, B.VIEW_COL5 INTEGER";
@@ -2413,7 +2450,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             // assert that the client cache for the base table is updated
             baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
             encodedCqCounter = baseTable.getEncodedCQCounter();
-            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 8), encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 8) : null, encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
             
             // assert client cache for view
             PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
@@ -2421,14 +2458,14 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             assertNull("A view should always have the column qualifier counter as null", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
             
             // assert that the server side metadata for the base table and the view is also updated correctly.
-            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 8, true);
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 8);
             assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
             assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 4);
             assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL3", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5);
             assertEncodedCQValue("A", "VIEW_COL4", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 6);
             assertEncodedCQValue("B", "VIEW_COL5", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 7);
-            // Adding a column to the should increment the base table's sequence number too since we update the cq counters for column families.
-            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 3);
+            // adding a column to the should increment the base table's sequence number too since we update the cq counters for column families.
+            assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 3 : baseTableSeqNumBeforeAddingChildCols );
             assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
             
             // Add column to the base table which doesn't already exist in the view.
@@ -2438,7 +2475,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             
             // assert that the client cache for the base table is updated 
             encodedCqCounter = baseTable.getEncodedCQCounter();
-            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 10), encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 10) : null, encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
             
             // assert client cache for view
             view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
@@ -2446,17 +2483,17 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             assertNull("A view should always have the column qualifier counter as null", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
             
             // assert that the server side metadata for the base table and the view is also updated correctly.
-            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 10), true);
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 10));
             assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL10", schemaName, viewName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 8));
             assertEncodedCQValue("A", "COL11", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 9);
-            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 4);
+            assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 4 : initBaseTableSeqNumber + 2 );
             assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 2);
         }
     }
 	
 	private void assertEncodedCQValue(String columnFamily, String columnName, String schemaName, String tableName, int expectedValue) throws Exception {
         String query = "SELECT " + COLUMN_QUALIFIER + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
-                + " = ? " + " AND " + COLUMN_FAMILY + " = ?" + " AND " + COLUMN_NAME  + " = ?";
+                + " = ? " + " AND " + COLUMN_FAMILY + " = ?" + " AND " + COLUMN_NAME  + " = ?" + " AND " + COLUMN_QUALIFIER  + " IS NOT NULL";
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             PreparedStatement stmt = conn.prepareStatement(query);
             stmt.setString(1, schemaName);
@@ -2465,12 +2502,16 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             stmt.setString(4, columnName);
             ResultSet rs = stmt.executeQuery();
             assertTrue(rs.next());
-            assertTrue(Bytes.equals(QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS.encode(expectedValue), rs.getBytes(1)));
+            if (columnEncoded) {
+                assertTrue(Bytes.equals(QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS.encode(expectedValue), rs.getBytes(1)));
+            } else {
+                assertTrue(Bytes.equals(columnName.getBytes(), rs.getBytes(1)));
+            }
             assertFalse(rs.next());
         }
     }
     
-    private void assertEncodedCQCounter(String columnFamily, String schemaName, String tableName, int expectedValue, boolean rowExists) throws Exception {
+    private void assertEncodedCQCounter(String columnFamily, String schemaName, String tableName, int expectedValue) throws Exception {
         String query = "SELECT " + COLUMN_QUALIFIER_COUNTER + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
                 + " = ? " + " AND " + COLUMN_FAMILY + " = ? AND " + COLUMN_QUALIFIER_COUNTER + " IS NOT NULL";
         try (Connection conn = DriverManager.getConnection(getUrl())) {
@@ -2479,7 +2520,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             stmt.setString(2, tableName);
             stmt.setString(3, columnFamily);
             ResultSet rs = stmt.executeQuery();
-            if (rowExists) {
+            if (columnEncoded) {
                 assertTrue(rs.next());
                 assertEquals(expectedValue, rs.getInt(1));
                 assertFalse(rs.next());
@@ -2511,7 +2552,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"ID VARCHAR(15) NOT NULL,\n"
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID))";
+                +"CONSTRAINT PK PRIMARY KEY (ID)) " + tableDDLOptions;
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         assertImmutableRows(conn, dataTableFullName, false);
@@ -2527,7 +2568,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"ID VARCHAR(15) NOT NULL,\n"
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID)) IMMUTABLE_ROWS=true";
+                +"CONSTRAINT PK PRIMARY KEY (ID)) " + generateDDLOptions("COLUMN_ENCODED_BYTES=4, IMMUTABLE_ROWS=true");
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         assertImmutableRows(conn, dataTableFullName, true);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/61d9035c/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index 310071f..73b3756 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -33,7 +32,6 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Collection;
-import java.util.Properties;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -47,8 +45,6 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -58,22 +54,41 @@ import org.junit.runners.Parameterized.Parameters;
 public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     
     private final boolean isMultiTenant;
+    private final boolean columnEncoded;
     
     private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
     private final String TENANT_SPECIFIC_URL2 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant2";
     
-    public AlterTableWithViewsIT(boolean isMultiTenant) {
+    public AlterTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
         this.isMultiTenant = isMultiTenant;
+        this.columnEncoded = columnEncoded;
     }
     
-    @Parameters(name="AlterTableWithViewsIT_multiTenant={0}") // name is used by failsafe as file name in reports
-    public static Collection<Boolean> data() {
-        return Arrays.asList(false, true);
+    @Parameters(name="AlterTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] { 
+                { false, false }, { false, true },
+                { true, false }, { true, true } });
     }
     
     private String generateDDL(String format) {
+        return generateDDL("", format);
+    }
+    
+    private String generateDDL(String options, String format) {
+        StringBuilder optionsBuilder = new StringBuilder(options);
+        if (columnEncoded) {
+            if (optionsBuilder.length()!=0)
+                optionsBuilder.append(",");
+            optionsBuilder.append("COLUMN_ENCODED_BYTES=4");
+        }
+        if (isMultiTenant) {
+            if (optionsBuilder.length()!=0)
+                optionsBuilder.append(",");
+            optionsBuilder.append("MULTI_TENANT=true");
+        }
         return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
-            isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");
+            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
     }
     
     @Test
@@ -96,7 +111,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             // adding a new pk column and a new regular column
             conn.createStatement().execute("ALTER TABLE " + tableName + " ADD COL3 varchar(10) PRIMARY KEY, COL4 integer");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 7, 5, "ID", "COL1", "COL2", "COL3", "COL4", "VIEW_COL1", "VIEW_COL2");
         } 
     }
@@ -113,28 +128,27 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                             + " COL1 integer NOT NULL,"
                             + " COL2 bigint NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
-                            + " ) UPDATE_CACHE_FREQUENCY=15 "
-                            + (isMultiTenant ? ",%s" : "%s");
-            conn.createStatement().execute(generateDDL(ddlFormat));
+                            + " ) %s ";
+            conn.createStatement().execute(generateDDL("UPDATE_CACHE_FREQUENCY=2", ddlFormat));
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
-            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 5");
+            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 1");
             
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             PTable table = phoenixConn.getTable(new PTableKey(null, tableName));
             PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
             assertFalse(table.isImmutableRows());
-            assertEquals(15, table.getUpdateCacheFrequency());
+            assertEquals(2, table.getUpdateCacheFrequency());
             PTable viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
             assertFalse(viewTable1.isImmutableRows());
-            assertEquals(15, viewTable1.getUpdateCacheFrequency());
+            assertEquals(2, viewTable1.getUpdateCacheFrequency());
             // query the view to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2);
             PTable viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
             assertFalse(viewTable2.isImmutableRows());
-            assertEquals(5, viewTable2.getUpdateCacheFrequency());
+            assertEquals(1, viewTable2.getUpdateCacheFrequency());
             
-            conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=10");
+            conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=3");
             // query the views to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable1);
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2);
@@ -142,16 +156,16 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             phoenixConn = conn.unwrap(PhoenixConnection.class);
             table = phoenixConn.getTable(new PTableKey(null, tableName));
             assertTrue(table.isImmutableRows());
-            assertEquals(10, table.getUpdateCacheFrequency());
+            assertEquals(3, table.getUpdateCacheFrequency());
             
             viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
             assertTrue(viewTable1.isImmutableRows());
-            assertEquals(10, viewTable1.getUpdateCacheFrequency());
+            assertEquals(3, viewTable1.getUpdateCacheFrequency());
             
             viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
             assertTrue(viewTable2.isImmutableRows());
             // update cache frequency is not propagated to the view since it was altered on the view
-            assertEquals(5, viewTable2.getUpdateCacheFrequency());
+            assertEquals(1, viewTable2.getUpdateCacheFrequency());
         } 
     }
     
@@ -178,7 +192,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
 
             // drop two columns from the base table
             conn.createStatement().execute("ALTER TABLE " + tableName + " DROP COLUMN COL3, COL5");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 4,
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 4,
                 QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL4");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 6, 4,
                 "ID", "COL1", "COL2", "COL4", "VIEW_COL1", "VIEW_COL2");
@@ -222,80 +236,84 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             viewConn.commit();
             
             try {
-                // adding a key value column to the base table that already exists in the view is not allowed
-                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+                // should fail because there is already a view column with same name of different type
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 char(10)");
                 fail();
-            } catch (SQLException e) {
+            }
+            catch (SQLException e) {
                 assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }           
+            
+            try {
+                // should fail because there is already a view column with same name with different scale
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,1)");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            } 
+            
+            try {
+                // should fail because there is already a view column with same name with different length
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(9,2)");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            } 
+            
+            try {
+                // should fail because there is already a view column with different length
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }
+            
+            // validate that there were no columns added to the table or view, if its table is column encoded the sequence number changes when we increment the cq counter
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 1 : 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+            
+            if (columnEncoded) {
+                try {
+                    // adding a key value column to the base table that already exists in the view is not allowed
+                    conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+                    fail();
+                } catch (SQLException e) {
+                    assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+                }
+            }
+            else {
+                // should succeed 
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+                assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
+                assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
+            
+                // query table
+                ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
+                assertTrue(rs.next());
+                assertEquals("view1", rs.getString("ID"));
+                assertEquals(12, rs.getInt("COL1"));
+                assertEquals(13, rs.getInt("COL2"));
+                assertEquals("view5", rs.getString("VIEW_COL2"));
+                assertEquals(17, rs.getInt("VIEW_COL4"));
+                assertFalse(rs.next());
+    
+                // query view
+                rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
+                assertTrue(rs.next());
+                assertEquals("view1", rs.getString("ID"));
+                assertEquals(12, rs.getInt("COL1"));
+                assertEquals(13, rs.getInt("COL2"));
+                assertEquals(14, rs.getInt("VIEW_COL1"));
+                assertEquals("view5", rs.getString("VIEW_COL2"));
+                assertEquals("view6", rs.getString("VIEW_COL3"));
+                assertEquals(17, rs.getInt("VIEW_COL4"));
+                assertEquals(18, rs.getInt("VIEW_COL5"));
+                assertEquals("view9", rs.getString("VIEW_COL6"));
+                assertFalse(rs.next());
             }
-//            try {
-//                // should fail because there is already a view column with same name of different type
-//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 char(10)");
-//                fail();
-//            }
-//            catch (SQLException e) {
-//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-//            }           
-//            
-//            try {
-//                // should fail because there is already a view column with same name with different scale
-//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,1)");
-//                fail();
-//            }
-//            catch (SQLException e) {
-//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-//            } 
-//            
-//            try {
-//                // should fail because there is already a view column with same name with different length
-//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(9,2)");
-//                fail();
-//            }
-//            catch (SQLException e) {
-//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-//            } 
-//            
-//            try {
-//                // should fail because there is already a view column with different length
-//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR");
-//                fail();
-//            }
-//            catch (SQLException e) {
-//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-//            }
-//            
-//            // validate that there were no columns added to the table or view
-//            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
-//            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
-//            
-//            // should succeed 
-//            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
-//            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
-//            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
-//            
-//            // query table
-//            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
-//            assertTrue(rs.next());
-//            assertEquals("view1", rs.getString("ID"));
-//            assertEquals(12, rs.getInt("COL1"));
-//            assertEquals(13, rs.getInt("COL2"));
-//            assertEquals("view5", rs.getString("VIEW_COL2"));
-//            assertEquals(17, rs.getInt("VIEW_COL4"));
-//            assertFalse(rs.next());
-//
-//            // query view
-//            rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
-//            assertTrue(rs.next());
-//            assertEquals("view1", rs.getString("ID"));
-//            assertEquals(12, rs.getInt("COL1"));
-//            assertEquals(13, rs.getInt("COL2"));
-//            assertEquals(14, rs.getInt("VIEW_COL1"));
-//            assertEquals("view5", rs.getString("VIEW_COL2"));
-//            assertEquals("view6", rs.getString("VIEW_COL3"));
-//            assertEquals(17, rs.getInt("VIEW_COL4"));
-//            assertEquals(18, rs.getInt("VIEW_COL5"));
-//            assertEquals("view9", rs.getString("VIEW_COL6"));
-//            assertFalse(rs.next());
         } 
     }
     
@@ -614,9 +632,9 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testAlteringViewThatHasChildViews() throws Exception {
-        String baseTable = "testAlteringViewThatHasChildViews";
-        String childView = "childView";
-        String grandChildView = "grandChildView";
+        String baseTable = generateUniqueName();
+        String childView = baseTable + "cildView";
+        String grandChildView = baseTable + "grandChildView";
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTable + "  ("

http://git-wip-us.apache.org/repos/asf/phoenix/blob/61d9035c/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java
new file mode 100644
index 0000000..dd64428
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java
@@ -0,0 +1,95 @@
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+
+public class ColumnEncodedBytesPropIT extends ParallelStatsDisabledIT {
+	
+	private String generateColsDDL(int numCols) {
+		StringBuilder sb = new StringBuilder();
+		for (int i=0; i<numCols; ++i) {
+			if (i>0) {
+				sb.append(" , ");
+			}
+			sb.append("col_").append(i).append(" VARCHAR ");
+		}
+		return sb.toString();
+	}
+	
+	@Test
+	public void testValidateProperty() throws SQLException {
+		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTableFullName1 = SchemaUtil.getTableName("", generateUniqueName());
+        String dataTableFullName2 = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            String ddl = "CREATE TABLE  " + dataTableFullName1 +
+                    "  (id varchar not null, val varchar " + 
+                    "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=4";
+            stmt.execute(ddl);
+            
+            ddl = "CREATE TABLE  " + dataTableFullName2 +
+                    "  (id varchar not null, val varchar " + 
+                    "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=NONE";
+            stmt.execute(ddl);
+            
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable dataTable1 = phxConn.getTable(new PTableKey(null, dataTableFullName1));
+            assertEquals("Encoding scheme set incorrectly", QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS, dataTable1.getEncodingScheme());
+            
+            PTable dataTable2 = phxConn.getTable(new PTableKey(null, dataTableFullName2));
+            assertEquals("Encoding scheme set incorrectly", QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, dataTable2.getEncodingScheme());
+        } 
+	}
+
+	@Test
+	public void testValidateMaxCols() throws SQLException {
+		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create table with more cols than is supported by the encoding
+            try {
+                stmt.execute("CREATE TABLE  " + dataTableFullName +
+                        "  (id varchar not null, " + generateColsDDL(QualifierEncodingScheme.ONE_BYTE_QUALIFIERS.getMaxQualifier()-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+2) + 
+                        "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=1");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.MAX_COLUMNS_EXCEEDED.getErrorCode(), e.getErrorCode());
+            }
+            
+            // create table with number of cols equal to that supported by the encoding
+            stmt.execute("CREATE TABLE  " + dataTableFullName +
+                    "  (id varchar not null, " + generateColsDDL(QualifierEncodingScheme.ONE_BYTE_QUALIFIERS.getMaxQualifier()-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1) + 
+                    "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=1");
+            
+            // add one more column
+            try {
+                stmt.execute("ALTER TABLE  " + dataTableFullName + " ADD val_x VARCHAR");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.MAX_COLUMNS_EXCEEDED.getErrorCode(), e.getErrorCode());
+            }
+        } 
+	}
+	
+}