You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/04 22:14:17 UTC

[50/50] [abbrv] phoenix git commit: Fail-fast iterators for EncodedColumnQualifierCellsList. Use list iterators instead of get(index) for navigating lists. Use HBase bytes utility for encoded column names. Fix test failures for immutable tables and index

Fail-fast iterators for EncodedColumnQualifierCellsList.
Use list iterators instead of get(index) for navigating lists.
Use HBase bytes utility for encoded column names.
Fix test failures for immutable tables and indexes.


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

Branch: refs/heads/encodecolumns2
Commit: ede568e9c4e4d35e7f4afe19637c8dd7cf5af23c
Parents: 87421ed
Author: Samarth <sa...@salesforce.com>
Authored: Wed Oct 5 00:11:07 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Fri Nov 4 15:12:54 2016 -0700

----------------------------------------------------------------------
 .../AlterMultiTenantTableWithViewsIT.java       |   25 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |  286 +++-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  143 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |    5 +
 .../org/apache/phoenix/end2end/OrderByIT.java   |    2 -
 .../phoenix/end2end/PhoenixRuntimeIT.java       |    4 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |    2 +-
 .../phoenix/end2end/StatsCollectorIT.java       |   16 +-
 .../apache/phoenix/end2end/StoreNullsIT.java    |   41 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   45 +-
 .../phoenix/end2end/index/DropMetadataIT.java   |   13 +-
 .../end2end/index/IndexExpressionIT.java        |   28 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |   26 +-
 .../phoenix/end2end/index/IndexTestUtil.java    |   11 +-
 .../end2end/index/MutableIndexFailureIT.java    |    2 +
 .../phoenix/compile/CreateTableCompiler.java    |   15 +-
 .../phoenix/compile/ExpressionCompiler.java     |   18 +-
 .../apache/phoenix/compile/FromCompiler.java    |   50 +-
 .../apache/phoenix/compile/JoinCompiler.java    |    8 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |    2 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |   11 +-
 .../compile/PostLocalIndexDDLCompiler.java      |    7 +-
 .../phoenix/compile/ProjectionCompiler.java     |   10 +-
 .../apache/phoenix/compile/QueryCompiler.java   |    2 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    2 +-
 .../compile/TupleProjectionCompiler.java        |   21 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    5 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |    5 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   13 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   40 +-
 .../coprocessor/DelegateRegionScanner.java      |    5 +
 .../GroupedAggregateRegionObserver.java         |   27 +-
 .../coprocessor/HashJoinRegionScanner.java      |    9 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  270 ++--
 .../phoenix/coprocessor/ScanRegionObserver.java |   15 +-
 .../UngroupedAggregateRegionObserver.java       |   16 +-
 .../coprocessor/generated/PTableProtos.java     | 1379 ++++++++++++++++--
 .../apache/phoenix/execute/BaseQueryPlan.java   |   25 +-
 .../apache/phoenix/execute/MutationState.java   |   14 +-
 .../apache/phoenix/execute/TupleProjector.java  |    6 +-
 .../expression/ArrayColumnExpression.java       |  142 ++
 .../expression/ArrayConstructorExpression.java  |    2 +-
 .../phoenix/expression/ExpressionType.java      |    5 +-
 .../expression/KeyValueColumnExpression.java    |   17 +-
 .../phoenix/expression/LiteralExpression.java   |   11 +-
 .../expression/ProjectedColumnExpression.java   |    1 +
 .../visitor/CloneExpressionVisitor.java         |    6 +
 .../expression/visitor/ExpressionVisitor.java   |    2 +
 .../StatelessTraverseAllExpressionVisitor.java  |    7 +-
 .../StatelessTraverseNoExpressionVisitor.java   |    7 +-
 .../phoenix/filter/ColumnProjectionFilter.java  |   24 +-
 .../filter/MultiKeyValueComparisonFilter.java   |    5 +-
 .../SingleCQKeyValueComparisonFilter.java       |    3 +-
 .../filter/SingleKeyValueComparisonFilter.java  |    4 +-
 .../apache/phoenix/hbase/index/ValueGetter.java |    1 +
 .../example/CoveredColumnIndexCodec.java        |    1 -
 .../hbase/index/util/KeyValueBuilder.java       |    1 +
 .../apache/phoenix/index/IndexMaintainer.java   |  327 ++++-
 .../phoenix/index/PhoenixIndexBuilder.java      |    2 +-
 .../index/PhoenixIndexFailurePolicy.java        |    5 +-
 .../index/PhoenixTransactionalIndexer.java      |   16 +-
 .../phoenix/iterate/BaseResultIterators.java    |   95 +-
 .../iterate/LookAheadResultIterator.java        |    2 +-
 .../phoenix/iterate/MappedByteBufferQueue.java  |    1 +
 .../phoenix/iterate/OrderedResultIterator.java  |    3 +-
 .../iterate/RegionScannerResultIterator.java    |   14 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   12 +-
 .../apache/phoenix/jdbc/PhoenixResultSet.java   |    2 +-
 .../apache/phoenix/join/HashCacheFactory.java   |    1 +
 .../mapreduce/FormatToBytesWritableMapper.java  |   22 +-
 .../mapreduce/FormatToKeyValueReducer.java      |   30 +-
 .../query/ConnectionQueryServicesImpl.java      |    4 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    1 -
 .../apache/phoenix/query/QueryConstants.java    |   56 +-
 .../phoenix/query/QueryServicesOptions.java     |    1 -
 .../org/apache/phoenix/schema/ColumnRef.java    |   22 +-
 .../apache/phoenix/schema/DelegateColumn.java   |    4 +
 .../apache/phoenix/schema/DelegateTable.java    |   19 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  348 ++++-
 .../java/org/apache/phoenix/schema/PColumn.java |    4 +-
 .../apache/phoenix/schema/PColumnFamily.java    |   14 +-
 .../phoenix/schema/PColumnFamilyImpl.java       |   50 +-
 .../org/apache/phoenix/schema/PColumnImpl.java  |   24 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    4 +-
 .../java/org/apache/phoenix/schema/PName.java   |   26 +
 .../java/org/apache/phoenix/schema/PTable.java  |  139 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  418 ++++--
 .../org/apache/phoenix/schema/PTableKey.java    |    6 +-
 .../apache/phoenix/schema/ProjectedColumn.java  |    1 +
 .../org/apache/phoenix/schema/SaltingUtil.java  |    2 +-
 .../apache/phoenix/schema/TableProperty.java    |    2 +-
 .../apache/phoenix/schema/tuple/BaseTuple.java  |   39 +
 .../phoenix/schema/tuple/DelegateTuple.java     |    7 +
 .../tuple/EncodedColumnQualiferCellsList.java   |  569 ++++++++
 .../schema/tuple/MultiKeyValueTuple.java        |    1 +
 .../tuple/PositionBasedMultiKeyValueTuple.java  |   87 ++
 .../schema/tuple/PositionBasedResultTuple.java  |  126 ++
 .../phoenix/schema/tuple/ResultTuple.java       |   20 +-
 .../org/apache/phoenix/schema/tuple/Tuple.java  |    4 +
 .../apache/phoenix/util/EncodedColumnsUtil.java |  108 ++
 .../java/org/apache/phoenix/util/IndexUtil.java |   77 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |    2 -
 .../org/apache/phoenix/util/MetaDataUtil.java   |    6 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |    8 +-
 .../org/apache/phoenix/util/ResultUtil.java     |   60 -
 .../java/org/apache/phoenix/util/ScanUtil.java  |   39 +
 .../org/apache/phoenix/util/SchemaUtil.java     |   54 +-
 .../phoenix/compile/HavingCompilerTest.java     |    2 +-
 .../phoenix/compile/QueryCompilerTest.java      |    6 +-
 .../phoenix/compile/WhereCompilerTest.java      |   27 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   10 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   10 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |    5 +-
 .../expression/ColumnExpressionTest.java        |   16 +-
 .../phoenix/index/IndexMaintainerTest.java      |    3 +-
 .../iterate/AggregateResultScannerTest.java     |    2 +-
 .../query/BaseConnectionlessQueryTest.java      |   18 +-
 .../phoenix/query/ConnectionlessTest.java       |    2 -
 .../EncodedColumnQualifierCellsListTest.java    |  608 ++++++++
 .../java/org/apache/phoenix/util/TestUtil.java  |   43 +-
 phoenix-protocol/src/main/PTable.proto          |    8 +
 121 files changed, 5485 insertions(+), 1022 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/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 adadca7..8275f3f 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,6 +17,7 @@
  */
 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;
@@ -476,14 +477,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");
             
-            // Add existing column VIEW_COL2 to the base table
+            // Adding existing column VIEW_COL2 to the base table isn't allowed.
             alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
-            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");
+            try {
+                conn.createStatement().execute(alterBaseTable);
+                fail();
+            } catch (SQLException e) {
+                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }
         }
     }
     
@@ -500,13 +501,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the both columns were added to view1
@@ -530,13 +531,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the dropped columns aren't visible
@@ -569,7 +570,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";
+                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR, NEWCOL5 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/ede568e9/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 48f4217..276390a 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
@@ -18,6 +18,15 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.hadoop.hbase.HColumnDescriptor.DEFAULT_REPLICATION_SCOPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODED_COLUMN_QUALIFIER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
+import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.closeConnection;
 import static org.apache.phoenix.util.TestUtil.closeStatement;
@@ -53,6 +62,7 @@ import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.EncodedCQCounter;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.IndexUtil;
@@ -231,8 +241,18 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals(exists, rs.next());
     }
 
+    
+    @Test
+    public void testDropIndexedColumnImmutableIndex() throws Exception {
+        helpTestDropIndexedColumn(true);
+    }
+    
     @Test
-    public void testDropIndexedColumn() throws Exception {
+    public void testDropIndexedColumnMutableIndex() throws Exception {
+        helpTestDropIndexedColumn(false);
+    }
+    
+    private void helpTestDropIndexedColumn(boolean immutable) throws Exception {
         String query;
         ResultSet rs;
         PreparedStatement stmt;
@@ -244,7 +264,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) " + (immutable ? "IMMUTABLE_ROWS = true" : ""));
         query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
@@ -2118,8 +2138,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             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.
-            assertTrue(table.getColumn("PK1").isRowTimestamp());
-            assertFalse(table.getColumn("PK2").isRowTimestamp());
+            assertTrue(table.getPColumnForColumnName("PK1").isRowTimestamp());
+            assertFalse(table.getPColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName, "PK1");
             
             String dataTableName2 = BaseTest.generateUniqueName();
@@ -2127,18 +2147,17 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute("CREATE TABLE " + dataTableFullName2 + " (PK1 VARCHAR, PK2 DATE PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR, KV2 INTEGER)");
             table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName2));
             // Assert that the column shows up as row time stamp in the cache.
-            assertFalse(table.getColumn("PK1").isRowTimestamp());
-            assertTrue(table.getColumn("PK2").isRowTimestamp());
+            assertFalse(table.getPColumnForColumnName("PK1").isRowTimestamp());
+            assertTrue(table.getPColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName2, "PK2");
             
             // Create an index on a table has a row time stamp pk column. The column should show up as a row time stamp column for the index too. 
             conn.createStatement().execute("CREATE INDEX " + indexTableName + "  ON " + dataTableFullName2 + " (KV1) include (KV2)");
             PTable indexTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), indexTableFullName));
-            String indexColName = IndexUtil.getIndexColumnName(table.getColumn("PK2"));
+            String indexColName = IndexUtil.getIndexColumnName(table.getPColumnForColumnName("PK2"));
             // Assert that the column shows up as row time stamp in the cache.
-            assertTrue(indexTable.getColumn(indexColName).isRowTimestamp());
+            assertTrue(indexTable.getPColumnForColumnName(indexColName).isRowTimestamp());
             assertIsRowTimestampSet(schemaName, indexTableName, indexColName);
-            
             String viewTableName2 = dataTableName2 + "_VIEW";
             String viewTableFullName2 = SchemaUtil.getTableName(schemaName, viewTableName2);
             // Creating a view with a row_timestamp column in its pk constraint is not allowed
@@ -2207,5 +2226,254 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 		}
 	}
 	
+	@Test
+	public void testMetadataForImmutableTableWithEncodedColumns() throws Exception {
+	    String schemaName = "XYZ";
+	    String baseTableName = generateUniqueName();
+	    String viewName = generateUniqueName();
+	    String fullTableName = schemaName + "." + baseTableName;
+	    String fullViewName = schemaName + "." + viewName;
+	    try (Connection conn = DriverManager.getConnection(getUrl())) {
+	        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+	        conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + fullTableName + " ("
+	                + " ID char(1) NOT NULL,"
+	                + " COL1 integer NOT NULL,"
+	                + " COL2 bigint NOT NULL,"
+	                + " KV1 VARCHAR"
+	                + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+	                + " )  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));
+	        
+	        
+	        // assert that the server side metadata is updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "KV1", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber);
+
+	        // now create a view and validate client and server side metadata
+	        String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
+	        conn.createStatement().execute(viewDDL);
+	        baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+	        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"));
+	        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);
+	        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, viewName, PTable.INITIAL_SEQ_NUM);
+	    }
+	}
+	
+	@Test
+	public void testMetadataForMutableTableWithEncodedColumns() throws Exception {
+	    String schemaName = "XYZ";
+	    String baseTableName = generateUniqueName();
+	    String viewName = generateUniqueName();
+	    String fullTableName = schemaName + "." + baseTableName;
+	    String fullViewName = schemaName + "." + viewName;
+	    try (Connection conn = DriverManager.getConnection(getUrl())) {
+	        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+	        conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + fullTableName + " ("
+	                + " ID char(1) NOT NULL,"
+	                + " COL1 integer NOT NULL,"
+	                + " COL2 bigint NOT NULL,"
+	                + " KV1 VARCHAR"
+	                + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+	                + " )");
+	        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));
+
+
+	        // assert that the server side metadata is updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "KV1", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber);
+
+	        // now create a view and validate client and server side metadata
+	        String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
+	        conn.createStatement().execute(viewDDL);
+	        baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+	        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));
+	        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);
+	        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, viewName, PTable.INITIAL_SEQ_NUM);
+	    }
+	}
+	
+	@Test
+    public void testAddingColumnsToTablesAndViewsWithEncodedColumns() throws Exception {
+        String schemaName = "XYZ";
+        String baseTableName = generateUniqueName();
+        String viewName = generateUniqueName();
+        String fullTableName = schemaName + "." + baseTableName;
+        String fullViewName = schemaName + "." + viewName;
+        Properties props = new Properties();
+        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + " ("
+                    + " ID char(1) NOT NULL,"
+                    + " COL1 integer NOT NULL,"
+                    + " COL2 bigint NOT NULL,"
+                    + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+                    + " )");
+            PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+            long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
+
+            // Add a column to the base table and see if the client and server metadata is updated correctly
+            String alterDDL = "ALTER TABLE " + fullTableName + " ADD COL3 VARCHAR PRIMARY KEY, COL4 INTEGER, COL5 VARCHAR, B.COL6 DECIMAL (10, 2)";
+            conn.createStatement().execute(alterDDL);
+
+            // 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));
+            
+            // assert that the server side metadata is updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3, true);
+            
+            // 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);
+
+            // 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);
+            
+            // 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);
+            
+
+            // Add column to the view
+            viewDDL = "ALTER VIEW " + fullViewName + " ADD VIEW_COL3 DECIMAL(10, 2), A.VIEW_COL4 VARCHAR, B.VIEW_COL5 INTEGER";
+            conn.createStatement().execute(viewDDL);
+
+            // 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));
+            
+            // assert client cache for view
+            PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+            encodedCqCounter = view.getEncodedCQCounter();
+            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);
+            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);
+            assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
+            
+            // Add column to the base table which doesn't already exist in the view.
+            alterDDL = "ALTER TABLE " + fullTableName + " ADD COL10 VARCHAR, A.COL11 INTEGER";
+            conn.createStatement().execute(alterDDL);
+            baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+            
+            // 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));
+            
+            // assert client cache for view
+            view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+            encodedCqCounter = view.getEncodedCQCounter();
+            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);
+            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, viewName, PTable.INITIAL_SEQ_NUM + 2);
+        }
+    }
+	
+	private void assertEncodedCQValue(String columnFamily, String columnName, String schemaName, String tableName, int expectedValue) throws Exception {
+        String query = "SELECT " + ENCODED_COLUMN_QUALIFIER + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
+                + " = ? " + " AND " + COLUMN_FAMILY + " = ?" + " AND " + COLUMN_NAME  + " = ?";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, schemaName);
+            stmt.setString(2, tableName);
+            stmt.setString(3, columnFamily);
+            stmt.setString(4, columnName);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(expectedValue, rs.getInt(1));
+            assertFalse(rs.next());
+        }
+    }
+    
+    private void assertEncodedCQCounter(String columnFamily, String schemaName, String tableName, int expectedValue, boolean rowExists) 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())) {
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, schemaName);
+            stmt.setString(2, tableName);
+            stmt.setString(3, columnFamily);
+            ResultSet rs = stmt.executeQuery();
+            if (rowExists) {
+                assertTrue(rs.next());
+                assertEquals(expectedValue, rs.getInt(1));
+                assertFalse(rs.next());
+            } else {
+                assertFalse(rs.next());
+            }
+        }
+    }
+    
+    private void assertSequenceNumber(String schemaName, String tableName, long expectedSequenceNumber) throws Exception {
+        String query = "SELECT " + TABLE_SEQ_NUM + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
+                + " = ? AND " +  TABLE_SEQ_NUM + " IS NOT NULL AND " + COLUMN_NAME + " IS NULL AND "
+                + COLUMN_FAMILY + " IS NULL ";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, schemaName);
+            stmt.setString(2, tableName);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(expectedSequenceNumber, rs.getInt(1));
+            assertFalse(rs.next());
+        }
+    }
+	
 }
  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/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 e6bf2d2..d0bc2c0 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
@@ -65,7 +65,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public static Collection<Boolean> data() {
         return Arrays.asList(false, true);
     }
-	
+    
     private String generateDDL(String format) {
         return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
             isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");
@@ -91,7 +91,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, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 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");
         } 
     }
@@ -119,7 +119,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, 1, 4,
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 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");
@@ -163,73 +163,80 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             viewConn.commit();
             
             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)");
+                // 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());
-            } 
-            
-            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) {
+            } 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, 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");
-            
-            // should succeed 
-            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 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());
         } 
     }
     
@@ -679,4 +686,4 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
         } 
     }
     
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
index 30bdb41..382fb06 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CreateTableIT.java
@@ -502,4 +502,9 @@ public class CreateTableIT extends BaseClientManagedTimeIT {
             fail();
         }
     }
+    
+    @Test
+    public void testCreateTableIfNotExistsForEncodedColumnNames() throws Exception {
+        
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
index 457b38e..dccbb12 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/OrderByIT.java
@@ -288,7 +288,6 @@ public class OrderByIT extends ParallelStatsDisabledIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
-
         try {
             String tableName1 = generateUniqueName();
             String ddl = "CREATE TABLE " + tableName1 +
@@ -376,7 +375,6 @@ public class OrderByIT extends ParallelStatsDisabledIT {
             assertEquals("a",rs.getString(1));  
             assertEquals(40,rs.getInt(2));
             assertFalse(rs.next()); 
-        } catch (SQLException e) {
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
index cddca04..91e9370 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PhoenixRuntimeIT.java
@@ -61,11 +61,11 @@ public class PhoenixRuntimeIT extends ParallelStatsDisabledIT {
         scan.setFilter(filter);
         ResultScanner scanner = htable.getScanner(scan);
         Result result = null;
-        ResultTuple tuple = new ResultTuple();
+        ResultTuple tuple;
         Set<String> actualTenantIds = Sets.newHashSetWithExpectedSize(tenantIds.length);
         Set<String> expectedTenantIds = new HashSet<>(Arrays.asList(tenantIds));
         while ((result = scanner.next()) != null) {
-            tuple.setResult(result);
+            tuple = new ResultTuple(result);
             e.evaluate(tuple, ptr);
             String tenantId = (String)PVarchar.INSTANCE.toObject(ptr);
             actualTenantIds.add(tenantId == null ? "" : tenantId);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
index f35484d..861455e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/RowValueConstructorIT.java
@@ -107,7 +107,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
                 count++;
             }
             // we have 6 values for a_integer present in the atable where a >= 4. x_integer is null for a_integer = 4. So the query should have returned 5 rows.
-            assertTrue(count == 5);   
+            assertEquals(5, count);   
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
index 6193cad..b13162b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StatsCollectorIT.java
@@ -130,13 +130,13 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
         explainPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
-                "CLIENT 4-CHUNK 1 ROWS 34 BYTES PARALLEL 3-WAY FULL SCAN OVER " + fullTableName + "\n" +
+                "CLIENT 4-CHUNK 1 ROWS 28 BYTES PARALLEL 3-WAY FULL SCAN OVER " + fullTableName + "\n" +
                 "CLIENT MERGE SORT",
                 explainPlan);
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName + " WHERE k = 'a'");
         explainPlan = QueryUtil.getExplainPlan(rs);
         assertEquals(
-                "CLIENT 1-CHUNK 1 ROWS 202 BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + fullTableName + "\n" +
+                "CLIENT 1-CHUNK 1 ROWS 204 BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + fullTableName + "\n" +
                 "CLIENT MERGE SORT",
                 explainPlan);
         
@@ -445,7 +445,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         List<KeyRange> keyRanges = getAllSplits(conn, fullTableName);
         assertEquals(26, keyRanges.size());
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        assertEquals("CLIENT 26-CHUNK 25 ROWS 12420 BYTES PARALLEL 1-WAY FULL SCAN OVER " + fullTableName,
+        assertEquals("CLIENT 26-CHUNK 25 ROWS 12530 BYTES PARALLEL 1-WAY FULL SCAN OVER " + fullTableName,
                 QueryUtil.getExplainPlan(rs));
 
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
@@ -468,25 +468,25 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
         assertTrue(rs.next());
         assertEquals("A", rs.getString(1));
         assertEquals(24, rs.getInt(2));
-        assertEquals(12144, rs.getInt(3));
+        assertEquals(12252, rs.getInt(3));
         assertEquals(11, rs.getInt(4));
 
         assertTrue(rs.next());
         assertEquals("B", rs.getString(1));
         assertEquals(20, rs.getInt(2));
-        assertEquals(5540, rs.getInt(3));
+        assertEquals(5600, rs.getInt(3));
         assertEquals(5, rs.getInt(4));
 
         assertTrue(rs.next());
         assertEquals("C", rs.getString(1));
         assertEquals(24, rs.getInt(2));
-        assertEquals(6652, rs.getInt(3));
+        assertEquals(6724, rs.getInt(3));
         assertEquals(6, rs.getInt(4));
 
         assertTrue(rs.next());
         assertEquals("D", rs.getString(1));
         assertEquals(24, rs.getInt(2));
-        assertEquals(6652, rs.getInt(3));
+        assertEquals(6724, rs.getInt(3));
         assertEquals(6, rs.getInt(4));
 
         assertFalse(rs.next());
@@ -531,7 +531,7 @@ public class StatsCollectorIT extends ParallelStatsEnabledIT {
             int startIndex = r.nextInt(strings.length);
             int endIndex = r.nextInt(strings.length - startIndex) + startIndex;
             long rows = endIndex - startIndex;
-            long c2Bytes = rows * 35;
+            long c2Bytes = rows * 37;
             rs = conn.createStatement().executeQuery(
                     "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH) from SYSTEM.STATS where PHYSICAL_NAME = '"
                             + fullTableName + "' AND GUIDE_POST_KEY>= cast('" + strings[startIndex]

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
index bb13f1b..07fc4e9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_EMPTY_COLUMN_BYTES;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
@@ -32,13 +34,22 @@ import java.util.Properties;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
@@ -52,6 +63,8 @@ import org.junit.Test;
  * functionality allows having row-level versioning (similar to how KEEP_DELETED_CELLS works), but
  * also allows permanently deleting a row.
  */
+
+//TODO: samarth  parameterize this test once the storage scheme is optional
 public class StoreNullsIT extends ParallelStatsDisabledIT {
     private static final Log LOG = LogFactory.getLog(StoreNullsIT.class);
     
@@ -95,31 +108,37 @@ public class StoreNullsIT extends ParallelStatsDisabledIT {
     }
 
     @Test
-    public void testStoringNulls() throws SQLException, InterruptedException, IOException {
+    public void testStoringNullsForImmutableTables() throws Exception {
         stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITH_NULLS + " VALUES (1, 'v1')");
         stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITHOUT_NULLS + " VALUES (1, 'v1')");
         stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITH_NULLS + " VALUES (2, null)");
         stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITHOUT_NULLS + " VALUES (2, null)");
 
-        ensureNullsNotStored(IMMUTABLE_WITH_NULLS);
-        ensureNullsNotStored(IMMUTABLE_WITHOUT_NULLS);
+        ensureNullsStoredAsEmptyByteArrays(IMMUTABLE_WITH_NULLS);
+        ensureNullsStoredAsEmptyByteArrays(IMMUTABLE_WITHOUT_NULLS);
     }
 
-    private void ensureNullsNotStored(String tableName) throws IOException {
-        tableName = SchemaUtil.normalizeIdentifier(tableName);
+    private void ensureNullsStoredAsEmptyByteArrays(String tableName) throws Exception {
         HTable htable = new HTable(getUtility().getConfiguration(), tableName);
         Scan s = new Scan();
         s.setRaw(true);
         ResultScanner scanner = htable.getScanner(s);
         // first row has a value for name
         Result rs = scanner.next();
-        assertTrue(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("NAME")));
-        assertTrue(rs.size() == 2);
-        // 2nd row has not
+        assertTrue(rs.size() == 2); // 2 because it also includes the empty key value column
+        PColumn nameColumn = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName)).getPColumnForColumnName("NAME");
+        ArrayColumnExpression colExpression = new ArrayColumnExpression(nameColumn, "NAME", true);
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        colExpression.evaluate(new ResultTuple(rs), ptr);
+        assertEquals(new ImmutableBytesPtr(PVarchar.INSTANCE.toBytes("v1")), ptr);
+        
         rs = scanner.next();
-        assertFalse(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("NAME")));
-        // and no delete marker either
-        assertTrue(rs.size() == 1);
+        assertTrue(rs.size() == 2); // 2 because it also includes the empty key value column
+        
+        // assert null stored as empty 
+        ptr = new ImmutableBytesPtr();
+        colExpression.evaluate(new ResultTuple(rs), ptr);
+        assertEquals(new ImmutableBytesPtr(ByteUtil.EMPTY_BYTE_ARRAY), ptr);
         assertNull(scanner.next());
         scanner.close();
         htable.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
index 498c4a3..8396639 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertValuesIT.java
@@ -35,15 +35,26 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Time;
 import java.sql.Timestamp;
+import java.util.Map.Entry;
+import java.util.NavigableMap;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.BaseTest;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.DateUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Test;
 
@@ -831,7 +842,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
             assertEquals("KV2", rs.getString(2));
             assertFalse(rs.next());
             
-            // Verify now that the data was correctly added to the mutable index too.
+            // Verify now that the data was correctly added to the immutable index too.
             stmt = conn.prepareStatement("SELECT KV2 FROM " + tableName + " WHERE PK2 = ? AND KV1 = ?");
             stmt.setDate(1, upsertedDate);
             stmt.setString(2, "KV1");
@@ -944,6 +955,38 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         }
     }
     
+    public void testColumnQualifierForUpsertedValues() throws Exception {
+        String schemaName = "A";
+        String tableName = "TEST";
+        String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        String ddl = "create table " + fullTableName 
+                + " (" 
+                + " K varchar primary key,"
+                + " CF1.V1 varchar, CF2.V2 VARCHAR, CF2.V3 VARCHAR)";
+        try (Connection conn = getConnection(nextTimestamp())) {
+            conn.createStatement().execute(ddl);
+        }
+        String dml = "UPSERT INTO " + fullTableName + " VALUES (?, ?, ?, ?)";
+        try (Connection conn = getConnection(nextTimestamp())) {
+            PreparedStatement stmt = conn.prepareStatement(dml);
+            stmt.setString(1, "KEY1");
+            stmt.setString(2, "VALUE1");
+            stmt.setString(3, "VALUE2");
+            stmt.setString(4, "VALUE3");
+            stmt.executeUpdate();
+            conn.commit();
+        }
+        // Issue a raw hbase scan and assert that key values have the expected column qualifiers.
+        try (Connection conn = getConnection(nextTimestamp())) {
+            HTableInterface table = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(fullTableName));
+            ResultScanner scanner = table.getScanner(new Scan());
+            Result next = scanner.next();
+            assertTrue(next.containsColumn(Bytes.toBytes("CF1"), PInteger.INSTANCE.toBytes(1)));
+            assertTrue(next.containsColumn(Bytes.toBytes("CF2"), PInteger.INSTANCE.toBytes(2)));
+            assertTrue(next.containsColumn(Bytes.toBytes("CF2"), PInteger.INSTANCE.toBytes(3)));
+        }
+    }
+    
     private static Connection getConnection(long ts) throws SQLException {
         Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
index 4e7d06a..b892c4d 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropMetadataIT.java
@@ -43,11 +43,13 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
@@ -175,6 +177,7 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
             
             // verify that the local index physical table was *not* dropped
             conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(localIndexTablePhysicalName.getBytes());
+            PTable localIndex2 = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, localIndexTableName2));
             
             // there should be a single row belonging to localIndexTableName2 
             Scan scan = new Scan();
@@ -184,7 +187,7 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
             Result result = results.next();
             assertNotNull(result);
             assertNotNull("localIndexTableName2 row is missing", result.getValue(QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES, 
-                IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V1").getBytes()));
+                EncodedColumnsUtil.getColumnQualifier(localIndex2.getPColumnForColumnName(IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V1")), localIndex2)));
             assertNull(results.next());
         }
     }
@@ -295,9 +298,11 @@ public class DropMetadataIT extends ParallelStatsDisabledIT {
             ResultScanner results = table.getScanner(scan);
             Result result = results.next();
             assertNotNull(result);
-            // there should be a single row belonging to " + viewIndex2 + " 
-            assertNotNull( viewIndex2 + " row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, 
-                IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V4").getBytes()));
+            PTable viewIndexPTable = pconn.getTable(new PTableKey(pconn.getTenantId(), viewIndex2));
+            PColumn column = viewIndexPTable.getPColumnForColumnName(IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V4"));
+            byte[] cq = EncodedColumnsUtil.getColumnQualifier(column, viewIndexPTable);
+            // there should be a single row belonging to VIEWINDEX2 
+            assertNotNull(viewIndex2 + " row is missing", result.getValue(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, cq));
             assertNull(results.next());
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
index 50548bd..ee8afa8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexExpressionIT.java
@@ -413,7 +413,12 @@ public class IndexExpressionIT extends ParallelStatsDisabledIT {
             rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexTableName);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
-
+            
+            String sql = "SELECT LONG_COL1 from " + fullDataTableName + " WHERE LONG_COL2 = 2";
+            rs = conn.createStatement().executeQuery(sql);
+            assertTrue(rs.next());
+            assertFalse(rs.next());
+            
             String dml = "DELETE from " + fullDataTableName + " WHERE long_col2 = 2";
             assertEquals(1, conn.createStatement().executeUpdate(dml));
             conn.commit();
@@ -870,8 +875,10 @@ public class IndexExpressionIT extends ParallelStatsDisabledIT {
 	        conn.setAutoCommit(false);
 	
 	        // make sure that the tables are empty, but reachable
-	        conn.createStatement().execute(
-	          "CREATE TABLE " + dataTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+            conn.createStatement().execute(
+                "CREATE TABLE " + dataTableName
+                        + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)"
+                        + (!mutable ? " IMMUTABLE_ROWS=true" : ""));
 	        query = "SELECT * FROM " + dataTableName ;
 	        rs = conn.createStatement().executeQuery(query);
 	        assertFalse(rs.next());
@@ -1244,7 +1251,16 @@ public class IndexExpressionIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testViewUsesTableIndex() throws Exception {
+    public void testViewUsesMutableTableIndex() throws Exception {
+        helpTestViewUsesTableIndex(false);
+    }
+    
+    @Test
+    public void testViewUsesImmutableTableIndex() throws Exception {
+        helpTestViewUsesTableIndex(true);
+    }
+    
+    private void helpTestViewUsesTableIndex(boolean immutable) throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
         try 
         {
@@ -1253,7 +1269,7 @@ public class IndexExpressionIT extends ParallelStatsDisabledIT {
             String viewName = generateUniqueName();
             String indexName2 = generateUniqueName();
         	ResultSet rs;
-	        String ddl = "CREATE TABLE " + dataTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, s1 VARCHAR, s2 VARCHAR, s3 VARCHAR, s4 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2))";
+	        String ddl = "CREATE TABLE " + dataTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, s1 VARCHAR, s2 VARCHAR, s3 VARCHAR, s4 VARCHAR CONSTRAINT pk PRIMARY KEY (k1, k2)) " + (immutable ? "IMMUTABLE_ROWS = true" : "");
 	        conn.createStatement().execute(ddl);
 	        conn.createStatement().execute("CREATE INDEX " + indexName1 + " ON " + dataTableName + "(k2, s2, s3, s1)");
 	        conn.createStatement().execute("CREATE INDEX " + indexName2 + " ON " + dataTableName + "(k2, s2||'_'||s3, s1, s4)");
@@ -1350,7 +1366,7 @@ public class IndexExpressionIT extends ParallelStatsDisabledIT {
 		try {
 			conn.createStatement().execute(
 					"CREATE TABLE " + dataTableName + " (k VARCHAR NOT NULL PRIMARY KEY, v VARCHAR) "
-							+ (mutable ? "IMMUTABLE_ROWS=true" : ""));
+							+ (!mutable ? "IMMUTABLE_ROWS=true" : ""));
 			String query = "SELECT * FROM  " + dataTableName;
 			ResultSet rs = conn.createStatement().executeQuery(query);
 			assertFalse(rs.next());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
index cb4310b..521a317 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexIT.java
@@ -807,23 +807,23 @@ public class IndexIT extends ParallelStatsDisabledIT {
             stmt.execute();
             conn.commit();
 
-            // make sure the index is working as expected
-            query = "SELECT * FROM " + fullIndexName;
+            query = "SELECT /*+ NO_INDEX */ * FROM " + testTable;
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
-            assertEquals("x", rs.getString(1));
-            assertEquals("1", rs.getString(2));
-            assertEquals("a", rs.getString(3));
+            assertEquals("a", rs.getString(1));
+            assertEquals("x", rs.getString(2));
+            assertEquals("1", rs.getString(3));
             assertTrue(rs.next());
-            assertEquals("y", rs.getString(1));
-            assertEquals("2", rs.getString(2));
-            assertEquals("b", rs.getString(3));
+            assertEquals("b", rs.getString(1));
+            assertEquals("y", rs.getString(2));
+            assertEquals("2", rs.getString(3));
             assertTrue(rs.next());
-            assertEquals("z", rs.getString(1));
-            assertEquals("3", rs.getString(2));
-            assertEquals("c", rs.getString(3));
+            assertEquals("c", rs.getString(1));
+            assertEquals("z", rs.getString(2));
+            assertEquals("3", rs.getString(3));
             assertFalse(rs.next());
-
+            
+            // make sure the index is working as expected
             query = "SELECT * FROM " + testTable;
             rs = conn.createStatement().executeQuery("EXPLAIN " + query);
             if (localIndex) {
@@ -896,7 +896,7 @@ public class IndexIT extends ParallelStatsDisabledIT {
             } else {
                 assertEquals("CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + fullIndexName + " ['1']", QueryUtil.getExplainPlan(rs));
             }
-
+            
             rs = conn.createStatement().executeQuery(query);
             assertTrue(rs.next());
             assertEquals("a",rs.getString(1));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
index e854f23..79cd1ce 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -48,6 +47,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -115,7 +115,7 @@ public class IndexTestUtil {
         while ((hasValue = dataRowKeySchema.next(ptr, i, maxOffset)) != null) {
             if (hasValue) {
                 PColumn dataColumn = dataPKColumns.get(i);
-                PColumn indexColumn = indexTable.getColumn(IndexUtil.getIndexColumnName(dataColumn));
+                PColumn indexColumn = indexTable.getPColumnForColumnName(IndexUtil.getIndexColumnName(dataColumn));
                 coerceDataValueToIndexValue(dataColumn, indexColumn, ptr);
                 indexValues[indexColumn.getPosition()-indexOffset] = ptr.copyBytes();
             }
@@ -135,10 +135,11 @@ public class IndexTestUtil {
                     for (Cell kv : entry.getValue()) {
                         @SuppressWarnings("deprecation")
                         byte[] cq = kv.getQualifier();
-                        if (Bytes.compareTo(QueryConstants.EMPTY_COLUMN_BYTES, cq) != 0) {
+                        byte[] emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(dataTable).getFirst();
+                        if (Bytes.compareTo(emptyKVQualifier, cq) != 0) {
                             try {
-                                PColumn dataColumn = family.getColumn(cq);
-                                PColumn indexColumn = indexTable.getColumn(IndexUtil.getIndexColumnName(family.getName().getString(), dataColumn.getName().getString()));
+                                PColumn dataColumn = family.getPColumnForColumnQualifier(cq);
+                                PColumn indexColumn = indexTable.getPColumnForColumnName(IndexUtil.getIndexColumnName(family.getName().getString(), dataColumn.getName().getString()));
                                 ptr.set(kv.getValueArray(),kv.getValueOffset(),kv.getValueLength());
                                 coerceDataValueToIndexValue(dataColumn, indexColumn, ptr);
                                 indexValues[indexPKColumns.indexOf(indexColumn)-indexOffset] = ptr.copyBytes();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index b968c76..b8df966 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -277,6 +277,8 @@ public class MutableIndexFailureIT extends BaseOwnClusterIT {
             // verify index table has correct data
             validateDataWithIndex(conn, fullTableName, fullIndexName);
             validateDataWithIndex(conn, secondTableName, secondFullIndexName);
+        } finally {
+            FAIL_WRITE = false;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index 07df105..c986c28 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
@@ -31,6 +31,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
@@ -239,7 +240,7 @@ public class CreateTableCompiler {
         }
     }
     
-    private static class ViewWhereExpressionVisitor extends StatelessTraverseNoExpressionVisitor<Boolean> {
+    public static class ViewWhereExpressionVisitor extends StatelessTraverseNoExpressionVisitor<Boolean> {
         private boolean isUpdatable = true;
         private final PTable table;
         private int position;
@@ -318,13 +319,23 @@ public class CreateTableCompiler {
         @Override
         public Boolean visit(KeyValueColumnExpression node) {
             try {
-                this.position = table.getColumnFamily(node.getColumnFamily()).getColumn(node.getColumnName()).getPosition();
+                this.position = table.getColumnFamily(node.getColumnFamily()).getPColumnForColumnQualifier(node.getColumnQualifier()).getPosition();
             } catch (SQLException e) {
                 throw new RuntimeException(e); // Impossible
             }
             return Boolean.TRUE;
         }
         
+        @Override
+        public Boolean visit(ArrayColumnExpression node) {
+            try {
+                this.position = table.getColumnFamily(node.getColumnFamily()).getPColumnForColumnQualifier(node.getEncodedColumnQualifier()).getPosition();
+            } catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+            return Boolean.TRUE;
+        }
+        
     }
     private static class VarbinaryDatum implements PDatum {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/ede568e9/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
index c05918b..846e966 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ExpressionCompiler.java
@@ -117,12 +117,26 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.TypeMismatchException;
-import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PUnsignedTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -386,7 +400,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
     }
 
     protected void addColumn(PColumn column) {
-        context.getScan().addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+        EncodedColumnsUtil.setColumns(column, context.getCurrentTable().getTable(), context.getScan());
     }
 
     @Override