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/03/21 21:53:45 UTC

[3/3] phoenix git commit: Replace empty key value with unit tests passing

Replace empty key value with unit tests passing


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

Branch: refs/heads/encodecolumns
Commit: 42294c4ba60da17d22e95bae6b4013871749b025
Parents: be5191b
Author: Samarth <sa...@salesforce.com>
Authored: Mon Mar 21 13:52:53 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Mar 21 13:52:53 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |  22 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |  42 ++++
 .../phoenix/end2end/index/IndexTestUtil.java    |  12 +-
 .../phoenix/compile/CreateTableCompiler.java    |   2 +-
 .../phoenix/compile/ExpressionCompiler.java     |   2 +-
 .../apache/phoenix/compile/FromCompiler.java    |  28 ++-
 .../apache/phoenix/compile/JoinCompiler.java    |   7 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |   2 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |  11 +-
 .../phoenix/compile/ProjectionCompiler.java     |   9 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |   2 +-
 .../compile/TupleProjectionCompiler.java        |   6 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   5 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   6 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   3 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  41 +++-
 .../coprocessor/SequenceRegionObserver.java     |   1 +
 .../UngroupedAggregateRegionObserver.java       |   8 +-
 .../coprocessor/generated/PTableProtos.java     | 242 ++++++++++++++++---
 .../apache/phoenix/execute/MutationState.java   |   2 +-
 .../expression/KeyValueColumnExpression.java    |  13 +-
 .../expression/ProjectedColumnExpression.java   |   1 +
 .../phoenix/filter/ColumnProjectionFilter.java  |  24 +-
 .../filter/MultiKeyValueComparisonFilter.java   |   2 +-
 .../filter/SingleKeyValueComparisonFilter.java  |   2 +-
 .../apache/phoenix/index/IndexMaintainer.java   |  42 ++--
 .../index/PhoenixTransactionalIndexer.java      |  12 +-
 .../phoenix/iterate/BaseResultIterators.java    |   5 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   4 +
 .../phoenix/mapreduce/AbstractBulkLoadTool.java |   2 +-
 .../mapreduce/FormatToBytesWritableMapper.java  |  44 ++--
 .../mapreduce/FormatToKeyValueReducer.java      |  10 +-
 .../query/ConnectionQueryServicesImpl.java      |   1 +
 .../apache/phoenix/query/QueryConstants.java    |  14 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |   4 +-
 .../apache/phoenix/schema/DelegateColumn.java   |   5 +
 .../apache/phoenix/schema/DelegateTable.java    |  14 +-
 .../apache/phoenix/schema/MetaDataClient.java   | 100 ++++++--
 .../java/org/apache/phoenix/schema/PColumn.java |   7 +-
 .../apache/phoenix/schema/PColumnFamily.java    |  14 +-
 .../phoenix/schema/PColumnFamilyImpl.java       |  47 ++--
 .../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  |  41 +++-
 .../org/apache/phoenix/schema/PTableImpl.java   | 136 ++++++++---
 .../apache/phoenix/schema/ProjectedColumn.java  |   1 +
 .../org/apache/phoenix/schema/SaltingUtil.java  |   2 +-
 .../java/org/apache/phoenix/util/IndexUtil.java |   7 +-
 .../org/apache/phoenix/util/PhoenixRuntime.java |   8 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |  59 ++++-
 .../phoenix/compile/HavingCompilerTest.java     |   2 +-
 .../phoenix/compile/QueryCompilerTest.java      |   2 +-
 .../phoenix/compile/WhereCompilerTest.java      |  26 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   4 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |   4 +-
 .../expression/ColumnExpressionTest.java        |  16 +-
 .../phoenix/index/IndexMaintainerTest.java      |   3 +-
 .../iterate/AggregateResultScannerTest.java     |   6 +-
 .../query/BaseConnectionlessQueryTest.java      |  18 +-
 .../phoenix/query/ConnectionlessTest.java       |  15 +-
 .../java/org/apache/phoenix/util/TestUtil.java  |   4 +-
 phoenix-protocol/src/main/PTable.proto          |   2 +
 63 files changed, 933 insertions(+), 297 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/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 122ec16..a1313dc 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
@@ -36,7 +36,6 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.Collections;
-import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -45,7 +44,9 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeepDeletedCells;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
+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.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.exception.SQLExceptionCode;
@@ -56,7 +57,6 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.types.PInteger;
-import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -65,8 +65,6 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-
 /**
  *
  * A lot of tests in this class test HBase level properties. As a result,
@@ -2123,23 +2121,23 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
             PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class); 
             PTable table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), "T1"));
             // 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("T1", "PK1");
             
             conn.createStatement().execute("CREATE TABLE T6 (PK1 VARCHAR, PK2 DATE PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR, KV2 INTEGER)");
             table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), "T6"));
             // 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("T6", "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 T6_IDX ON T6 (KV1) include (KV2)");
             PTable indexTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), "T6_IDX"));
-            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("T6_IDX", indexColName);
             
             // Creating a view with a row_timestamp column in its pk constraint is not allowed
@@ -2204,6 +2202,6 @@ public class AlterTableIT extends BaseOwnClusterHBaseManagedTimeIT {
 			}
 		}
 	}
-    
+	
 }
  
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/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 78dce80..3fec718 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
@@ -37,12 +37,20 @@ import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.Properties;
 
+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.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;
 
@@ -922,6 +930,40 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
         }
     }
     
+    @Test
+    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(1)));
+            assertTrue(next.containsColumn(Bytes.toBytes("CF2"), PInteger.INSTANCE.toBytes(2)));
+        }
+    }
+    
+    
     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/42294c4b/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 ba04ad7..42c0264 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,15 +39,14 @@ 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;
-import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PRow;
 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.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -115,7 +114,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 +134,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 = SchemaUtil.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/42294c4b/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 e032feb..03f679b 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
@@ -290,7 +290,7 @@ 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
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/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 1278494..31a2f82 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
@@ -407,7 +407,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
             return LiteralExpression.newConstant(column.getDataType().toObject(ptr), column.getDataType());
         }
         if (tableRef.equals(context.getCurrentTable()) && !SchemaUtil.isPKColumn(column)) { // project only kv columns
-            context.getScan().addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+            context.getScan().addColumn(column.getFamilyName().getBytes(), SchemaUtil.getColumnQualifier(column, tableRef.getTable()));
         }
         Expression expression = ref.newColumnExpression(node.isTableNameCaseSensitive(), node.isCaseSensitive());
         Expression wrappedExpression = wrapGroupByExpression(expression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index ffe9621..8462812 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -122,10 +122,12 @@ public class FromCompiler {
             throw new ColumnNotFoundException(schemaName, tableName, null, colName);
         }
         
+        @Override
         public PFunction resolveFunction(String functionName) throws SQLException {
             throw new FunctionNotFoundException(functionName);
         };
 
+        @Override
         public boolean hasUDFs() {
             return false;
         };
@@ -158,7 +160,7 @@ public class FromCompiler {
                     if (htable != null) Closeables.closeQuietly(htable);
                 }
                 tableNode = NamedTableNode.create(null, baseTable, statement.getColumnDefs());
-                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1));
+                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1), false);
             }
             throw e;
         }
@@ -221,7 +223,7 @@ public class FromCompiler {
             Expression sourceExpression = projector.getColumnProjector(column.getPosition()).getExpression();
             PColumnImpl projectedColumn = new PColumnImpl(column.getName(), column.getFamilyName(),
                     sourceExpression.getDataType(), sourceExpression.getMaxLength(), sourceExpression.getScale(), sourceExpression.isNullable(),
-                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic());
+                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getColumnQualifier());
             projectedColumns.add(projectedColumn);
         }
         PTable t = PTableImpl.makePTable(table, projectedColumns);
@@ -258,12 +260,12 @@ public class FromCompiler {
     	private final List<TableRef> tableRefs;
     	private final String alias;
 
-       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes) throws SQLException  {
+       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes, boolean useEncodedColumnNames) throws SQLException  {
            super(connection, 0, false, udfParseNodes);
            List<PColumnFamily> families = Lists.newArrayListWithExpectedSize(table.getDynamicColumns().size());
            for (ColumnDef def : table.getDynamicColumns()) {
                if (def.getColumnDefName().getFamilyName() != null) {
-                   families.add(new PColumnFamilyImpl(PNameFactory.newName(def.getColumnDefName().getFamilyName()),Collections.<PColumn>emptyList()));
+                   families.add(new PColumnFamilyImpl(PNameFactory.newName(def.getColumnDefName().getFamilyName()),Collections.<PColumn>emptyList(), useEncodedColumnNames));
                }
            }
            Long scn = connection.getSCN();
@@ -360,8 +362,8 @@ public class FromCompiler {
 
 			}
         	PColumn column = resolveCF
-        	        ? tableRef.getTable().getColumnFamily(tableName).getColumn(colName)
-        			: tableRef.getTable().getColumn(colName);
+        	        ? tableRef.getTable().getColumnFamily(tableName).getPColumnForColumnName(colName)
+        			: tableRef.getTable().getPColumnForColumnName(colName);
             return new ColumnRef(tableRef, column.getPosition());
 		}
     }
@@ -546,8 +548,9 @@ public class FromCompiler {
                         theTable.getColumnFamily(family); // Verifies that column family exists
                         familyName = PNameFactory.newName(family);
                     }
+                    // Dynamic columns don't have a corresponding column qualifier
                     allcolumns.add(new PColumnImpl(name, familyName, dynColumn.getDataType(), dynColumn.getMaxLength(),
-                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true));
+                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true, null));
                     position++;
                 }
                 theTable = PTableImpl.makePTable(theTable, allcolumns);
@@ -643,16 +646,17 @@ public class FromCompiler {
                     // referenced by an outer wild-card select.
                     alias = String.valueOf(position);
                 }
+                //TODO: samarth confirm this is the right change i.e. using null for column qualifier
                 PColumnImpl column = new PColumnImpl(PNameFactory.newName(alias),
                         PNameFactory.newName(QueryConstants.DEFAULT_COLUMN_FAMILY),
-                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false);
+                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false, null);
                 columns.add(column);
             }
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,
                     PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                     null, null, columns, null, null, Collections.<PTable>emptyList(),
                     false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                    null, null, false, false, 0, 0L);
+                    null, null, false, false, 0, 0L, null);
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);
@@ -727,7 +731,7 @@ public class FromCompiler {
                 while (iterator.hasNext()) {
                     TableRef tableRef = iterator.next();
                     try {
-                        PColumn column = tableRef.getTable().getColumn(colName);
+                        PColumn column = tableRef.getTable().getPColumnForColumnName(colName);
                         if (theTableRef != null) { throw new AmbiguousColumnException(colName); }
                         theTableRef = tableRef;
                         theColumnPosition = column.getPosition();
@@ -740,12 +744,12 @@ public class FromCompiler {
             } else {
                 try {
                     TableRef tableRef = resolveTable(schemaName, tableName);
-                    PColumn column = tableRef.getTable().getColumn(colName);
+                    PColumn column = tableRef.getTable().getPColumnForColumnName(colName);
                     return new ColumnRef(tableRef, column.getPosition());
                 } catch (TableNotFoundException e) {
                     // Try using the tableName as a columnFamily reference instead
                     ColumnFamilyRef cfRef = resolveColumnFamily(schemaName, tableName);
-                    PColumn column = cfRef.getFamily().getColumn(colName);
+                    PColumn column = cfRef.getFamily().getPColumnForColumnName(colName);
                     return new ColumnRef(cfRef.getTableRef(), column.getPosition());
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 5d03f57..5f4adf6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -74,6 +74,7 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ProjectedColumn;
@@ -714,7 +715,7 @@ public class JoinCompiler {
                 if (columnRef.getTableRef().equals(tableRef)
                         && !SchemaUtil.isPKColumn(columnRef.getColumn())
                         && !(columnRef instanceof LocalIndexColumnRef)) {
-                    scan.addColumn(columnRef.getColumn().getFamilyName().getBytes(), columnRef.getColumn().getName().getBytes());
+                    scan.addColumn(columnRef.getColumn().getFamilyName().getBytes(), SchemaUtil.getColumnQualifier(columnRef.getColumn(), tableRef.getTable()));
                 }
             }
         }
@@ -1295,14 +1296,14 @@ public class JoinCompiler {
         if (left.getBucketNum() != null) {
             merged.remove(0);
         }
-        
+        //TODO: samarth should projected join table should always have non-encoded column names?
         return PTableImpl.makePTable(left.getTenantId(), left.getSchemaName(),
                 PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), right.getName().getString())),
                 left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(), 
                 left.getBucketNum(), merged,left.getParentSchemaName(), left.getParentTableName(), left.getIndexes(),
                 left.isImmutableRows(), Collections.<PName>emptyList(), null, null, PTable.DEFAULT_DISABLE_WAL,
                 left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexId(), left.getIndexType(),
-                left.rowKeyOrderOptimizable(), left.isTransactional(), left.getUpdateCacheFrequency(), left.getIndexDisableTimestamp());
+                left.rowKeyOrderOptimizable(), left.isTransactional(), left.getUpdateCacheFrequency(), left.getIndexDisableTimestamp(), StorageScheme.NON_ENCODED_COLUMN_NAMES);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
index 7f3277a..2a7c507 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ListJarsQueryPlan.java
@@ -80,7 +80,7 @@ public class ListJarsQueryPlan implements QueryPlan {
         PColumn column =
                 new PColumnImpl(PNameFactory.newName("jar_location"), null,
                         PVarchar.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false, false);
+                        false, null, false, false, null);
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
index 2659b3f..10087af 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
@@ -49,6 +49,7 @@ import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ScanUtil;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TransactionUtil;
 
 import com.google.common.collect.Lists;
@@ -164,8 +165,8 @@ public class PostDDLCompiler {
                             @Override
                             public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
                                 PColumn column = tableName != null
-                                        ? tableRef.getTable().getColumnFamily(tableName).getColumn(colName)
-                                        : tableRef.getTable().getColumn(colName);
+                                        ? tableRef.getTable().getColumnFamily(tableName).getPColumnForColumnName(colName)
+                                        : tableRef.getTable().getPColumnForColumnName(colName);
                                 return new ColumnRef(tableRef, column.getPosition());
                             }
                             
@@ -191,6 +192,7 @@ public class PostDDLCompiler {
                         ScanUtil.setTimeRange(scan, ts);
                         if (emptyCF != null) {
                             scan.setAttribute(BaseScannerRegionObserver.EMPTY_CF, emptyCF);
+                            scan.setAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER, SchemaUtil.getEmptyKeyValueInfo(tableRef.getTable()).getFirst());
                         }
                         ServerCache cache = null;
                         try {
@@ -214,11 +216,12 @@ public class PostDDLCompiler {
                                     // data empty column family to stay the same, while the index empty column family
                                     // changes.
                                     PColumn column = deleteList.get(0);
+                                    byte[] cq = SchemaUtil.getColumnQualifier(column, tableRef.getTable());
                                     if (emptyCF == null) {
-                                        scan.addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+                                        scan.addColumn(column.getFamilyName().getBytes(), cq);
                                     }
                                     scan.setAttribute(BaseScannerRegionObserver.DELETE_CF, column.getFamilyName().getBytes());
-                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_CQ, column.getName().getBytes());
+                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_CQ, cq);
                                 }
                             }
                             List<byte[]> columnFamilies = Lists.newArrayListWithExpectedSize(tableRef.getTable().getColumnFamilies().size());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index 3cf3934..b6a6771 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -217,7 +217,7 @@ public class ProjectionCompiler {
             PColumn indexColumn = null;
             ColumnRef ref = null;
             try {
-                indexColumn = index.getColumn(indexColName);
+                indexColumn = index.getPColumnForColumnName(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
             } catch (ColumnNotFoundException e) {
                 if (index.getIndexType() == IndexType.LOCAL) {
@@ -288,7 +288,7 @@ public class ProjectionCompiler {
             PColumn indexColumn = null;
             ColumnRef ref = null;
             try {
-                indexColumn = index.getColumn(indexColName);
+                indexColumn = index.getPColumnForColumnName(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
             } catch (ColumnNotFoundException e) {
                 if (index.getIndexType() == IndexType.LOCAL) {
@@ -476,7 +476,7 @@ public class ProjectionCompiler {
                 }
             } else {
                 for (byte[] cq : entry.getValue()) {
-                    PColumn column = family.getColumn(cq);
+                    PColumn column = family.getPColumnForColumnQualifier(cq);
                     Integer maxLength = column.getMaxLength();
                     int byteSize = column.getDataType().isFixedWidth() ? maxLength == null ? column.getDataType().getByteSize() : maxLength : RowKeySchema.ESTIMATED_VARIABLE_LENGTH_SIZE;
                     estimatedByteSize += SizedUtil.KEY_VALUE_SIZE + estimatedKeySize + byteSize;
@@ -702,7 +702,8 @@ public class ProjectionCompiler {
                      public Void visit(ProjectedColumnExpression expression) {
                          if (expression.getDataType().isArrayType()) {
                              indexProjectedColumns.add(expression);
-                             KeyValueColumnExpression keyValueColumnExpression = new KeyValueColumnExpression(expression.getColumn());
+                             //TODO: samarth confirm this change that column names 
+                             KeyValueColumnExpression keyValueColumnExpression = new KeyValueColumnExpression(expression.getColumn(), false);
                              indexKVs.add(keyValueColumnExpression);
                              copyOfChildren.set(0, keyValueColumnExpression);
                              Integer count = arrayExpressionCounts.get(expression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
index 58cdb64..de63249 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TraceQueryPlan.java
@@ -78,7 +78,7 @@ public class TraceQueryPlan implements QueryPlan {
         PColumn column =
                 new PColumnImpl(PNameFactory.newName(MetricInfo.TRACE.columnName), null,
                         PLong.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false, false);
+                        false, null, false, false, null);
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index 4be78a9..0933e34 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -136,7 +136,7 @@ public class TupleProjectionCompiler {
             projectedColumns.add(column);
             // Wildcard or FamilyWildcard will be handled by ProjectionCompiler.
             if (!isWildcard && !families.contains(sourceColumn.getFamilyName())) {
-                context.getScan().addColumn(sourceColumn.getFamilyName().getBytes(), sourceColumn.getName().getBytes());
+                context.getScan().addColumn(sourceColumn.getFamilyName().getBytes(), SchemaUtil.getColumnQualifier(column, table));
             }
         }
         // add LocalIndexDataColumnRef
@@ -152,7 +152,7 @@ public class TupleProjectionCompiler {
                 table.getBucketNum(), projectedColumns, table.getParentSchemaName(),
                 table.getParentName(), table.getIndexes(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp());
+                table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.getStorageScheme());
     }
 
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
@@ -179,7 +179,7 @@ public class TupleProjectionCompiler {
                     retainPKColumns ? table.getBucketNum() : null, projectedColumns, null,
                     null, Collections.<PTable>emptyList(), table.isImmutableRows(), Collections.<PName>emptyList(), null, null,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
-                    null, table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp());
+                    null, table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.getStorageScheme());
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index b25baf7..0331015 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -73,16 +73,17 @@ public class UnionCompiler {
             ColumnProjector colProj = plan.getProjector().getColumnProjector(i);
             Expression sourceExpression = colProj.getExpression();
             String name = selectNodes == null ? colProj.getName() : selectNodes.get(i).getAlias();
+            //TODO: samarth confirm this is the right change
             PColumnImpl projectedColumn = new PColumnImpl(PNameFactory.newName(name), UNION_FAMILY_NAME,
                     sourceExpression.getDataType(), sourceExpression.getMaxLength(), sourceExpression.getScale(), sourceExpression.isNullable(),
-                    i, sourceExpression.getSortOrder(), 500, null, false, sourceExpression.toString(), false, false);
+                    i, sourceExpression.getSortOrder(), 500, null, false, sourceExpression.toString(), false, false, null);
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();
         PTable tempTable = PTableImpl.makePTable(statement.getConnection().getTenantId(), UNION_SCHEMA_NAME, UNION_TABLE_NAME, 
                 PTableType.SUBQUERY, null, HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn, null, null,
                         projectedColumns, null, null, null,
-                        true, null, null, null, true, true, true, null, null, null, false, false, 0, 0L);
+                        true, null, null, null, true, true, true, null, null, null, false, false, 0, 0L, null);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);
         return tableRef;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 13963d7..c2e42f2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -170,8 +170,8 @@ public class WhereCompiler {
             TableRef tableRef = ref.getTableRef();
             if (tableRef.equals(context.getCurrentTable()) && !SchemaUtil.isPKColumn(ref.getColumn())) {
                 // track the where condition columns. Later we need to ensure the Scan in HRS scans these column CFs
-                context.addWhereCoditionColumn(ref.getColumn().getFamilyName().getBytes(), ref.getColumn().getName()
-                        .getBytes());
+                byte[] cq = SchemaUtil.getColumnQualifier(ref.getColumn(), tableRef.getTable());
+                context.addWhereCoditionColumn(ref.getColumn().getFamilyName().getBytes(), cq);
             }
             return ref.newColumnExpression(node.isTableNameCaseSensitive(), node.isCaseSensitive());
         }
@@ -194,7 +194,7 @@ public class WhereCompiler {
             // just use that.
             try {
                 if (!SchemaUtil.isPKColumn(ref.getColumn())) {
-                    table.getColumn(ref.getColumn().getName().getString());
+                    table.getPColumnForColumnName(ref.getColumn().getName().getString());
                 }
             } catch (AmbiguousColumnException e) {
                 disambiguateWithFamily = true;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index 9b440ac..af154fa 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -77,6 +77,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     public static final String DELETE_CQ = "_DeleteCQ";
     public static final String DELETE_CF = "_DeleteCF";
     public static final String EMPTY_CF = "_EmptyCF";
+    public static final String EMPTY_COLUMN_QUALIFIER = "_EmptyColumnQualifier";
     public static final String SPECIFIC_ARRAY_INDEX = "_SpecificArrayIndex";
     public static final String GROUP_BY_LIMIT = "_GroupByLimit";
     public static final String LOCAL_INDEX = "_LocalIndex";
@@ -399,7 +400,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                             Cell kv = tuple.getValue(idx);
                             if (Bytes.equals(kvExp.getColumnFamily(), 0, kvExp.getColumnFamily().length,
                                     kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength())
-                                && Bytes.equals(kvExp.getColumnName(), 0, kvExp.getColumnName().length,
+                                && Bytes.equals(kvExp.getColumnQualifier(), 0, kvExp.getColumnQualifier().length,
                                         kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())) {
                                 // remove the kv that has the full array values.
                                 result.remove(idx);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index bcfdf20..a2db5d6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -19,12 +19,14 @@ package org.apache.phoenix.coprocessor;
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
+import static org.apache.phoenix.schema.PTable.StorageScheme;
 import static org.apache.hadoop.hbase.KeyValue.createFirstOnRow;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE_BYTES;
@@ -54,6 +56,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORAGE_SCHEME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_INDEX;
@@ -178,6 +181,7 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PSmallint;
+import org.apache.phoenix.schema.types.PTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.trace.util.Tracing;
@@ -246,6 +250,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue ROW_KEY_ORDER_OPTIMIZABLE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ROW_KEY_ORDER_OPTIMIZABLE_BYTES);
     private static final KeyValue TRANSACTIONAL_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TRANSACTIONAL_BYTES);
     private static final KeyValue UPDATE_CACHE_FREQUENCY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, UPDATE_CACHE_FREQUENCY_BYTES);
+    private static final KeyValue STORAGE_SCHEME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, STORAGE_SCHEME_BYTES);
     
     private static final List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
             EMPTY_KEYVALUE_KV,
@@ -269,7 +274,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             BASE_COLUMN_COUNT_KV,
             ROW_KEY_ORDER_OPTIMIZABLE_KV,
             TRANSACTIONAL_KV,
-            UPDATE_CACHE_FREQUENCY_KV
+            UPDATE_CACHE_FREQUENCY_KV,
+            STORAGE_SCHEME_KV
             );
     static {
         Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
@@ -296,6 +302,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int TRANSACTIONAL_INDEX = TABLE_KV_COLUMNS.indexOf(TRANSACTIONAL_KV);
     private static final int UPDATE_CACHE_FREQUENCY_INDEX = TABLE_KV_COLUMNS.indexOf(UPDATE_CACHE_FREQUENCY_KV);
     private static final int INDEX_DISABLE_TIMESTAMP = TABLE_KV_COLUMNS.indexOf(INDEX_DISABLE_TIMESTAMP_KV);
+    private static final int STORAGE_SCHEME_INDEX = TABLE_KV_COLUMNS.indexOf(STORAGE_SCHEME_KV);
 
     // KeyValues for Column
     private static final KeyValue DECIMAL_DIGITS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
@@ -309,6 +316,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue IS_VIEW_REFERENCED_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_VIEW_REFERENCED_BYTES);
     private static final KeyValue COLUMN_DEF_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_DEF_BYTES);
     private static final KeyValue IS_ROW_TIMESTAMP_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_ROW_TIMESTAMP_BYTES);
+    private static final KeyValue COLUMN_QUALIFIER_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_QUALIFIER_BYTES);
     private static final List<KeyValue> COLUMN_KV_COLUMNS = Arrays.<KeyValue>asList(
             DECIMAL_DIGITS_KV,
             COLUMN_SIZE_KV,
@@ -321,7 +329,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             VIEW_CONSTANT_KV,
             IS_VIEW_REFERENCED_KV,
             COLUMN_DEF_KV,
-            IS_ROW_TIMESTAMP_KV
+            IS_ROW_TIMESTAMP_KV,
+            COLUMN_QUALIFIER_KV
             );
     static {
         Collections.sort(COLUMN_KV_COLUMNS, KeyValue.COMPARATOR);
@@ -337,6 +346,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int IS_VIEW_REFERENCED_INDEX = COLUMN_KV_COLUMNS.indexOf(IS_VIEW_REFERENCED_KV);
     private static final int COLUMN_DEF_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_DEF_KV);
     private static final int IS_ROW_TIMESTAMP_INDEX = COLUMN_KV_COLUMNS.indexOf(IS_ROW_TIMESTAMP_KV);
+    private static final int COLUMN_QUALIFIER_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_QUALIFIER_KV);
     
     private static final int LINK_TYPE_INDEX = 0;
 
@@ -642,8 +652,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 isRowTimestampKV == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
                         isRowTimestampKV.getValueArray(), isRowTimestampKV.getValueOffset(),
                         isRowTimestampKV.getValueLength()));
+        Cell columnQualifierKV = colKeyValues[COLUMN_QUALIFIER_INDEX];
+        Integer columnQualifier = columnQualifierKV == null ? null :
+            PInteger.INSTANCE.getCodec().decodeInt(columnQualifierKV.getValueArray(), columnQualifierKV.getValueOffset(), SortOrder.getDefault());
 
-        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false);
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false, columnQualifier);
         columns.add(column);
     }
     
@@ -840,7 +853,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Cell indexDisableTimestampKv = tableKeyValues[INDEX_DISABLE_TIMESTAMP];
         long indexDisableTimestamp = indexDisableTimestampKv == null ? 0L : PLong.INSTANCE.getCodec().decodeLong(indexDisableTimestampKv.getValueArray(),
                 indexDisableTimestampKv.getValueOffset(), SortOrder.getDefault());
-        
+        Cell storageSchemeKv = tableKeyValues[STORAGE_SCHEME_INDEX];
+        byte v = (byte)PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(), storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength());
+        StorageScheme storageScheme = storageSchemeKv == null ? null : StorageScheme.fromSerializedValue(v);
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = new ArrayList<PTable>();
         List<PName> physicalTables = new ArrayList<PName>();
@@ -885,7 +900,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             tableSeqNum, pkName, saltBucketNum, columns, tableType == INDEX ? schemaName : null,
             tableType == INDEX ? dataTableName : null, indexes, isImmutableRows, physicalTables, defaultFamilyName, viewStatement,
             disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency,
-            stats, baseColumnCount, indexDisableTimestamp);
+            stats, baseColumnCount, indexDisableTimestamp, storageScheme);
     }
 
     private PFunction getFunction(RegionScanner scanner, final boolean isReplace, long clientTimeStamp, List<Mutation> deleteMutationsForReplace)
@@ -1901,8 +1916,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
                 String columnFamily = rkmd[FAMILY_NAME_INDEX] == null ? null : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
                 try {
-                    existingViewColumn = columnFamily == null ? view.getColumn(columnName) : view.getColumnFamily(
-                            columnFamily).getColumn(columnName);
+                    existingViewColumn = columnFamily == null ? view.getPColumnForColumnName(columnName) : view.getColumnFamily(
+                            columnFamily).getPColumnForColumnName(columnName);
                 } catch (ColumnFamilyNotFoundException e) {
                     // ignore since it means that the column family is not present for the column to be added.
                 } catch (ColumnNotFoundException e) {
@@ -2352,7 +2367,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
                                     PColumnFamily family =
                                             table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
-                                    family.getColumn(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+                                    family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
                                 } else if (pkCount > COLUMN_NAME_INDEX
                                         && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
                                     addingPKColumn = true;
@@ -2600,7 +2615,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         PColumnFamily family =
                                                 table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
                                         columnToDelete =
-                                                family.getColumn(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+                                                family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
                                     } else if (pkCount > COLUMN_NAME_INDEX
                                             && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
                                         deletePKColumn = true;
@@ -2644,8 +2659,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                             byte[] indexKey =
                                                     SchemaUtil.getTableKey(tenantId, index
                                                             .getSchemaName().getBytes(), index.getTableName().getBytes());
+                                            byte[] cq = SchemaUtil.getColumnQualifier(columnToDelete, index);
                                             // If index requires this column for its pk, then drop it
-                                            if (indexColumns.contains(new ColumnReference(columnToDelete.getFamilyName().getBytes(), columnToDelete.getName().getBytes()))) {
+                                            if (indexColumns.contains(new ColumnReference(columnToDelete.getFamilyName().getBytes(), cq))) {
                                                 // Since we're dropping the index, lock it to ensure
                                                 // that a change in index state doesn't
                                                 // occur while we're dropping it.
@@ -2858,7 +2874,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         tableMetadata = new ArrayList<Mutation>(tableMetadata);
                         // insert an empty KV to trigger time stamp update on data table row
                         Put p = new Put(dataTableKey);
-                        p.add(TABLE_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, timeStamp, QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+                        // Decide on what column qualifier to use for empty key value.
+                        PTable currentTable = doGetTable(key, HConstants.LATEST_TIMESTAMP, rowLock);
+                        Pair<byte[], byte[]> emptyKeyValuePair = SchemaUtil.getEmptyKeyValueInfo(currentTable);
+                        p.add(TABLE_FAMILY_BYTES, emptyKeyValuePair.getFirst(), timeStamp, emptyKeyValuePair.getSecond());
                         tableMetadata.add(p);
                     }
                     boolean setRowKeyOrderOptimizableCell = newState == PIndexState.BUILDING && !rowKeyOrderOptimizable;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
index 4bc1ef5..d5fd96c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/SequenceRegionObserver.java
@@ -80,6 +80,7 @@ public class SequenceRegionObserver extends BaseRegionObserver {
     public static final String NUM_TO_ALLOCATE = "NUM_TO_ALLOCATE";
     private static final byte[] SUCCESS_VALUE = PInteger.INSTANCE.toBytes(Integer.valueOf(Sequence.SUCCESS));
     
+    //TODO: samarth verify that it is ok to send non-encoded empty column here. Probably is.
     private static Result getErrorResult(byte[] row, long timestamp, int errorCode) {
         byte[] errorCodeBuf = new byte[PInteger.INSTANCE.getByteSize()];
         PInteger.INSTANCE.getCodec().encodeInt(errorCode, errorCodeBuf, 0);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 5481910..3bdcee3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -231,6 +231,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         byte[] deleteCQ = null;
         byte[] deleteCF = null;
         byte[] emptyCF = null;
+        byte[] emptyKVQualifier = null;
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         if (upsertSelectTable != null) {
             isUpsert = true;
@@ -245,6 +246,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                 deleteCQ = scan.getAttribute(BaseScannerRegionObserver.DELETE_CQ);
             }
             emptyCF = scan.getAttribute(BaseScannerRegionObserver.EMPTY_CF);
+            emptyKVQualifier = scan.getAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER);
         }
         TupleProjector tupleProjector = null;
         HRegion dataRegion = null;
@@ -495,8 +497,10 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     if (!timeStamps.contains(kvts)) {
                                         Put put = new Put(kv.getRowArray(), kv.getRowOffset(),
                                             kv.getRowLength());
-                                        put.add(emptyCF, QueryConstants.EMPTY_COLUMN_BYTES, kvts,
-                                            ByteUtil.EMPTY_BYTE_ARRAY);
+                                        // FIXME: Use the right byte array value. Transactional tables can't
+                                        // have empty byte arrays since Tephra seems them as delete markers.
+                                        put.add(emptyCF, emptyKVQualifier != null ? emptyKVQualifier
+                                                : QueryConstants.EMPTY_COLUMN_BYTES, kvts, ByteUtil.EMPTY_BYTE_ARRAY);
                                         mutations.add(put);
                                     }
                                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 9fdfe51..83a7db9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -269,6 +269,16 @@ public final class PTableProtos {
      * <code>optional bool isDynamic = 14;</code>
      */
     boolean getIsDynamic();
+
+    // optional int32 columnQualifier = 15;
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    boolean hasColumnQualifier();
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    int getColumnQualifier();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -391,6 +401,11 @@ public final class PTableProtos {
               isDynamic_ = input.readBool();
               break;
             }
+            case 120: {
+              bitField0_ |= 0x00004000;
+              columnQualifier_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -709,6 +724,22 @@ public final class PTableProtos {
       return isDynamic_;
     }
 
+    // optional int32 columnQualifier = 15;
+    public static final int COLUMNQUALIFIER_FIELD_NUMBER = 15;
+    private int columnQualifier_;
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    public boolean hasColumnQualifier() {
+      return ((bitField0_ & 0x00004000) == 0x00004000);
+    }
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    public int getColumnQualifier() {
+      return columnQualifier_;
+    }
+
     private void initFields() {
       columnNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       familyNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -724,6 +755,7 @@ public final class PTableProtos {
       expression_ = "";
       isRowTimestamp_ = false;
       isDynamic_ = false;
+      columnQualifier_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -799,6 +831,9 @@ public final class PTableProtos {
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
         output.writeBool(14, isDynamic_);
       }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        output.writeInt32(15, columnQualifier_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -864,6 +899,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(14, isDynamic_);
       }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(15, columnQualifier_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -957,6 +996,11 @@ public final class PTableProtos {
         result = result && (getIsDynamic()
             == other.getIsDynamic());
       }
+      result = result && (hasColumnQualifier() == other.hasColumnQualifier());
+      if (hasColumnQualifier()) {
+        result = result && (getColumnQualifier()
+            == other.getColumnQualifier());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1026,6 +1070,10 @@ public final class PTableProtos {
         hash = (37 * hash) + ISDYNAMIC_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getIsDynamic());
       }
+      if (hasColumnQualifier()) {
+        hash = (37 * hash) + COLUMNQUALIFIER_FIELD_NUMBER;
+        hash = (53 * hash) + getColumnQualifier();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1163,6 +1211,8 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00001000);
         isDynamic_ = false;
         bitField0_ = (bitField0_ & ~0x00002000);
+        columnQualifier_ = 0;
+        bitField0_ = (bitField0_ & ~0x00004000);
         return this;
       }
 
@@ -1247,6 +1297,10 @@ public final class PTableProtos {
           to_bitField0_ |= 0x00002000;
         }
         result.isDynamic_ = isDynamic_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00004000;
+        }
+        result.columnQualifier_ = columnQualifier_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1309,6 +1363,9 @@ public final class PTableProtos {
         if (other.hasIsDynamic()) {
           setIsDynamic(other.getIsDynamic());
         }
+        if (other.hasColumnQualifier()) {
+          setColumnQualifier(other.getColumnQualifier());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1909,6 +1966,39 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int32 columnQualifier = 15;
+      private int columnQualifier_ ;
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public boolean hasColumnQualifier() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public int getColumnQualifier() {
+        return columnQualifier_;
+      }
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public Builder setColumnQualifier(int value) {
+        bitField0_ |= 0x00004000;
+        columnQualifier_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public Builder clearColumnQualifier() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        columnQualifier_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PColumn)
     }
 
@@ -3338,6 +3428,16 @@ public final class PTableProtos {
      * <code>optional int64 indexDisableTimestamp = 29;</code>
      */
     long getIndexDisableTimestamp();
+
+    // optional bytes storageScheme = 30;
+    /**
+     * <code>optional bytes storageScheme = 30;</code>
+     */
+    boolean hasStorageScheme();
+    /**
+     * <code>optional bytes storageScheme = 30;</code>
+     */
+    com.google.protobuf.ByteString getStorageScheme();
   }
   /**
    * Protobuf type {@code PTable}
@@ -3553,6 +3653,11 @@ public final class PTableProtos {
               indexDisableTimestamp_ = input.readInt64();
               break;
             }
+            case 242: {
+              bitField0_ |= 0x02000000;
+              storageScheme_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4163,6 +4268,22 @@ public final class PTableProtos {
       return indexDisableTimestamp_;
     }
 
+    // optional bytes storageScheme = 30;
+    public static final int STORAGESCHEME_FIELD_NUMBER = 30;
+    private com.google.protobuf.ByteString storageScheme_;
+    /**
+     * <code>optional bytes storageScheme = 30;</code>
+     */
+    public boolean hasStorageScheme() {
+      return ((bitField0_ & 0x02000000) == 0x02000000);
+    }
+    /**
+     * <code>optional bytes storageScheme = 30;</code>
+     */
+    public com.google.protobuf.ByteString getStorageScheme() {
+      return storageScheme_;
+    }
+
     private void initFields() {
       schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       tableNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -4193,6 +4314,7 @@ public final class PTableProtos {
       transactional_ = false;
       updateCacheFrequency_ = 0L;
       indexDisableTimestamp_ = 0L;
+      storageScheme_ = com.google.protobuf.ByteString.EMPTY;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4347,6 +4469,9 @@ public final class PTableProtos {
       if (((bitField0_ & 0x01000000) == 0x01000000)) {
         output.writeInt64(29, indexDisableTimestamp_);
       }
+      if (((bitField0_ & 0x02000000) == 0x02000000)) {
+        output.writeBytes(30, storageScheme_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4477,6 +4602,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(29, indexDisableTimestamp_);
       }
+      if (((bitField0_ & 0x02000000) == 0x02000000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(30, storageScheme_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4633,6 +4762,11 @@ public final class PTableProtos {
         result = result && (getIndexDisableTimestamp()
             == other.getIndexDisableTimestamp());
       }
+      result = result && (hasStorageScheme() == other.hasStorageScheme());
+      if (hasStorageScheme()) {
+        result = result && getStorageScheme()
+            .equals(other.getStorageScheme());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4762,6 +4896,10 @@ public final class PTableProtos {
         hash = (37 * hash) + INDEXDISABLETIMESTAMP_FIELD_NUMBER;
         hash = (53 * hash) + hashLong(getIndexDisableTimestamp());
       }
+      if (hasStorageScheme()) {
+        hash = (37 * hash) + STORAGESCHEME_FIELD_NUMBER;
+        hash = (53 * hash) + getStorageScheme().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4944,6 +5082,8 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x08000000);
         indexDisableTimestamp_ = 0L;
         bitField0_ = (bitField0_ & ~0x10000000);
+        storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x20000000);
         return this;
       }
 
@@ -5104,6 +5244,10 @@ public final class PTableProtos {
           to_bitField0_ |= 0x01000000;
         }
         result.indexDisableTimestamp_ = indexDisableTimestamp_;
+        if (((from_bitField0_ & 0x20000000) == 0x20000000)) {
+          to_bitField0_ |= 0x02000000;
+        }
+        result.storageScheme_ = storageScheme_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5285,6 +5429,9 @@ public final class PTableProtos {
         if (other.hasIndexDisableTimestamp()) {
           setIndexDisableTimestamp(other.getIndexDisableTimestamp());
         }
+        if (other.hasStorageScheme()) {
+          setStorageScheme(other.getStorageScheme());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -7054,6 +7201,42 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional bytes storageScheme = 30;
+      private com.google.protobuf.ByteString storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes storageScheme = 30;</code>
+       */
+      public boolean hasStorageScheme() {
+        return ((bitField0_ & 0x20000000) == 0x20000000);
+      }
+      /**
+       * <code>optional bytes storageScheme = 30;</code>
+       */
+      public com.google.protobuf.ByteString getStorageScheme() {
+        return storageScheme_;
+      }
+      /**
+       * <code>optional bytes storageScheme = 30;</code>
+       */
+      public Builder setStorageScheme(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x20000000;
+        storageScheme_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes storageScheme = 30;</code>
+       */
+      public Builder clearStorageScheme() {
+        bitField0_ = (bitField0_ & ~0x20000000);
+        storageScheme_ = getDefaultInstance().getStorageScheme();
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PTable)
     }
 
@@ -7089,7 +7272,7 @@ public final class PTableProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\014PTable.proto\032\021PGuidePosts.proto\"\246\002\n\007PC" +
+      "\n\014PTable.proto\032\021PGuidePosts.proto\"\277\002\n\007PC" +
       "olumn\022\027\n\017columnNameBytes\030\001 \002(\014\022\027\n\017family" +
       "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \002(\t\022\021\n\tmax" +
       "Length\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010nullable\030" +
@@ -7097,32 +7280,33 @@ public final class PTableProtos {
       "(\005\022\021\n\tarraySize\030\t \001(\005\022\024\n\014viewConstant\030\n " +
       "\001(\014\022\026\n\016viewReferenced\030\013 \001(\010\022\022\n\nexpressio" +
       "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\022\021\n\tisDyn" +
-      "amic\030\016 \001(\010\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022" +
-      "\016\n\006values\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003",
-      " \001(\003\022\025\n\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePost" +
-      "sCount\030\005 \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGui" +
-      "dePosts\"\303\005\n\006PTable\022\027\n\017schemaNameBytes\030\001 " +
-      "\002(\014\022\026\n\016tableNameBytes\030\002 \002(\014\022\036\n\ttableType" +
-      "\030\003 \002(\0162\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022" +
-      "\026\n\016sequenceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002" +
-      "(\003\022\023\n\013pkNameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002" +
-      "(\005\022\031\n\007columns\030\t \003(\0132\010.PColumn\022\030\n\007indexes" +
-      "\030\n \003(\0132\007.PTable\022\027\n\017isImmutableRows\030\013 \002(\010" +
-      "\022 \n\nguidePosts\030\014 \003(\0132\014.PTableStats\022\032\n\022da",
-      "taTableNameBytes\030\r \001(\014\022\031\n\021defaultFamilyN" +
-      "ame\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTe" +
-      "nant\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewStat" +
-      "ement\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010te" +
-      "nantId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tind" +
-      "exType\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\n" +
-      "storeNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(" +
-      "\005\022\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtra" +
-      "nsactional\030\033 \001(\010\022\034\n\024updateCacheFrequency" +
-      "\030\034 \001(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003*A\n",
-      "\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIE" +
-      "W\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.p" +
-      "hoenix.coprocessor.generatedB\014PTableProt" +
-      "osH\001\210\001\001\240\001\001"
+      "amic\030\016 \001(\010\022\027\n\017columnQualifier\030\017 \001(\005\"\232\001\n\013" +
+      "PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006values\030\002 \003(\014",
+      "\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n\rkeyByte" +
+      "sCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001(\005\022!\n" +
+      "\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"\332\005\n\006PTa" +
+      "ble\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tableNam" +
+      "eBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.PTable" +
+      "Type\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenceNumb" +
+      "er\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pkNameByt" +
+      "es\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007columns\030\t" +
+      " \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007.PTable" +
+      "\022\027\n\017isImmutableRows\030\013 \002(\010\022 \n\nguidePosts\030",
+      "\014 \003(\0132\014.PTableStats\022\032\n\022dataTableNameByte" +
+      "s\030\r \001(\014\022\031\n\021defaultFamilyName\030\016 \001(\014\022\022\n\ndi" +
+      "sableWAL\030\017 \002(\010\022\023\n\013multiTenant\030\020 \002(\010\022\020\n\010v" +
+      "iewType\030\021 \001(\014\022\025\n\rviewStatement\030\022 \001(\014\022\025\n\r" +
+      "physicalNames\030\023 \003(\014\022\020\n\010tenantId\030\024 \001(\014\022\023\n" +
+      "\013viewIndexId\030\025 \001(\005\022\021\n\tindexType\030\026 \001(\014\022\026\n" +
+      "\016statsTimeStamp\030\027 \001(\003\022\022\n\nstoreNulls\030\030 \001(" +
+      "\010\022\027\n\017baseColumnCount\030\031 \001(\005\022\036\n\026rowKeyOrde" +
+      "rOptimizable\030\032 \001(\010\022\025\n\rtransactional\030\033 \001(" +
+      "\010\022\034\n\024updateCacheFrequency\030\034 \001(\003\022\035\n\025index",
+      "DisableTimestamp\030\035 \001(\003\022\025\n\rstorageScheme\030" +
+      "\036 \001(\014*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020" +
+      "\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org." +
+      "apache.phoenix.coprocessor.generatedB\014PT" +
+      "ableProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -7134,7 +7318,7 @@ public final class PTableProtos {
           internal_static_PColumn_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PColumn_descriptor,
-              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", });
+              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "ColumnQualifier", });
           internal_static_PTableStats_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_PTableStats_fieldAccessorTable = new
@@ -7146,7 +7330,7 @@ public final class PTableProtos {
           internal_static_PTable_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PTable_descriptor,
-              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "GuidePosts", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisableTimestamp", });
+              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "GuidePosts", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisableTimestamp", "StorageScheme", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index cc8f7a5..71af992 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -767,7 +767,7 @@ public class MutationState implements SQLCloseable {
                 }
                 for (PColumn column : columns) {
                     if (column != null) {
-                        resolvedTable.getColumnFamily(column.getFamilyName().getString()).getColumn(column.getName().getString());
+                        resolvedTable.getColumnFamily(column.getFamilyName().getString()).getPColumnForColumnName(column.getName().getString());
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
index 4b5fdbb..163cc72 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
@@ -46,8 +46,8 @@ public class KeyValueColumnExpression extends ColumnExpression {
     public KeyValueColumnExpression() {
     }
 
-    public KeyValueColumnExpression(PColumn column) {
-        this(column, null);
+    public KeyValueColumnExpression(PColumn column, boolean encodedColumnName) {
+        this(column, null, encodedColumnName);
     }
 
     public KeyValueColumnExpression(PDatum column, byte[] cf, byte[] cq) {
@@ -56,18 +56,19 @@ public class KeyValueColumnExpression extends ColumnExpression {
         this.cq = cq;
     }
 
-    public KeyValueColumnExpression(PColumn column, String displayName) {
+    public KeyValueColumnExpression(PColumn column, String displayName, boolean encodedColumnName) {
         super(column);
         this.cf = column.getFamilyName().getBytes();
-        this.cq = column.getName().getBytes();
+        this.cq = SchemaUtil.getColumnQualifier(column, encodedColumnName);
         this.displayName = displayName;
     }
 
     public byte[] getColumnFamily() {
         return cf;
     }
-
-    public byte[] getColumnName() {
+    
+    //TODO: samarth look for the callers of this.
+    public byte[] getColumnQualifier() {
         return cq;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/42294c4b/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
index 3a38dee..2744f35 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ProjectedColumnExpression.java
@@ -154,6 +154,7 @@ public class ProjectedColumnExpression extends ColumnExpression {
         return Determinism.PER_INVOCATION;
     }
 
+    @Override
     public ProjectedColumnExpression clone() {
         return new ProjectedColumnExpression(this.column, this.columns, this.position, this.displayName);
     }