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/07 21:23:07 UTC

[15/20] 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 indexes.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
index 15d6d2f..c5f690b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToKeyValueReducer.java
@@ -44,6 +44,7 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.Closeables;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -89,7 +90,7 @@ public class FormatToKeyValueReducer
     }
 
     private void initColumnsMap(PhoenixConnection conn) throws SQLException {
-        Map<byte[], Integer> indexMap = new TreeMap(Bytes.BYTES_COMPARATOR);
+        Map<byte[], Integer> indexMap = new TreeMap<>(Bytes.BYTES_COMPARATOR);
         columnIndexes = new HashMap<>();
         int columnIndex = 0;
         for (int index = 0; index < logicalNames.size(); index++) {
@@ -98,12 +99,16 @@ public class FormatToKeyValueReducer
             for (int i = 0; i < cls.size(); i++) {
                 PColumn c = cls.get(i);
                 byte[] family = new byte[0];
-                if (c.getFamilyName() != null) {
+                byte[] cq;
+                if (!SchemaUtil.isPKColumn(c)) {
                     family = c.getFamilyName().getBytes();
+                    cq = EncodedColumnsUtil.getColumnQualifier(c, table);
+                } else {
+                    // TODO: samarth verify if this is the right thing to do here.
+                    cq = c.getName().getBytes();
                 }
-                byte[] name = c.getName().getBytes();
-                byte[] cfn = Bytes.add(family, QueryConstants.NAMESPACE_SEPARATOR_BYTES, name);
-                Pair<byte[], byte[]> pair = new Pair(family, name);
+                byte[] cfn = Bytes.add(family, QueryConstants.NAMESPACE_SEPARATOR_BYTES, cq);
+                Pair<byte[], byte[]> pair = new Pair<>(family, cq);
                 if (!indexMap.containsKey(cfn)) {
                     indexMap.put(cfn, new Integer(columnIndex));
                     columnIndexes.put(new Integer(columnIndex), pair);
@@ -111,8 +116,8 @@ public class FormatToKeyValueReducer
                 }
             }
             byte[] emptyColumnFamily = SchemaUtil.getEmptyColumnFamily(table);
-            Pair<byte[], byte[]> pair = new Pair(emptyColumnFamily, QueryConstants
-                    .EMPTY_COLUMN_BYTES);
+            byte[] emptyKeyValue = EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
+            Pair<byte[], byte[]> pair = new Pair<>(emptyColumnFamily, emptyKeyValue);
             columnIndexes.put(new Integer(columnIndex), pair);
             columnIndex++;
         }
@@ -123,18 +128,17 @@ public class FormatToKeyValueReducer
                           Reducer<TableRowkeyPair, ImmutableBytesWritable, TableRowkeyPair, KeyValue>.Context context)
             throws IOException, InterruptedException {
         TreeSet<KeyValue> map = new TreeSet<KeyValue>(KeyValue.COMPARATOR);
-        ImmutableBytesWritable rowKey = key.getRowkey();
         for (ImmutableBytesWritable aggregatedArray : values) {
             DataInputStream input = new DataInputStream(new ByteArrayInputStream(aggregatedArray.get()));
             while (input.available() != 0) {
                 byte type = input.readByte();
                 int index = WritableUtils.readVInt(input);
                 ImmutableBytesWritable family;
-                ImmutableBytesWritable name;
+                ImmutableBytesWritable cq;
                 ImmutableBytesWritable value = QueryConstants.EMPTY_COLUMN_VALUE_BYTES_PTR;
                 Pair<byte[], byte[]> pair = columnIndexes.get(index);
                 family = new ImmutableBytesWritable(pair.getFirst());
-                name = new ImmutableBytesWritable(pair.getSecond());
+                cq = new ImmutableBytesWritable(pair.getSecond());
                 int len = WritableUtils.readVInt(input);
                 if (len > 0) {
                     byte[] array = new byte[len];
@@ -145,10 +149,10 @@ public class FormatToKeyValueReducer
                 KeyValue.Type kvType = KeyValue.Type.codeToType(type);
                 switch (kvType) {
                     case Put: // not null value
-                        kv = builder.buildPut(key.getRowkey(), family, name, value);
+                        kv = builder.buildPut(key.getRowkey(), family, cq, value);
                         break;
                     case DeleteColumn: // null value
-                        kv = builder.buildDeleteColumns(key.getRowkey(), family, name);
+                        kv = builder.buildDeleteColumns(key.getRowkey(), family, cq);
                         break;
                     default:
                         throw new IOException("Unsupported KeyValue type " + kvType);
@@ -164,4 +168,4 @@ public class FormatToKeyValueReducer
             if (++index % 100 == 0) context.setStatus("Wrote " + index);
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 3405564..61580c8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -97,6 +97,7 @@ import org.apache.hadoop.hbase.ipc.PhoenixRpcSchedulerFactory;
 import org.apache.hadoop.hbase.ipc.ServerRpcController;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.regionserver.IndexHalfStoreFileReaderGenerator;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
@@ -576,7 +577,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -4117,4 +4117,4 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public Configuration getConfiguration() {
         return config;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 6398a23..cf57e62 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -93,7 +93,6 @@ import org.apache.tephra.inmemory.InMemoryTxSystemClient;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
-
 /**
  *
  * Implementation of ConnectionQueryServices used in testing where no connection to

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 8e2dc1a..e44d83e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
@@ -31,6 +31,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
 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.COLUMN_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CURRENT_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CYCLE_FLAG;
@@ -40,6 +41,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODED_COLUMN_QUALIFIER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
@@ -85,6 +87,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SOURCE_DATA_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATA_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SQL_DATETIME_SUB;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.START_WITH;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORAGE_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
@@ -104,6 +107,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
 
 import java.math.BigDecimal;
 
@@ -118,7 +122,8 @@ import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 
 
 /**
@@ -149,23 +154,30 @@ public interface QueryConstants {
     public final static byte[] OFFSET_ROW_KEY_BYTES = Bytes.toBytes(OFFSET_ROW_KEY);
     public final static ImmutableBytesPtr OFFSET_ROW_KEY_PTR = new ImmutableBytesPtr(OFFSET_ROW_KEY_BYTES);
 
-    public final static PName SINGLE_COLUMN_NAME = PNameFactory.newNormalizedName("s");
-    public final static PName SINGLE_COLUMN_FAMILY_NAME = PNameFactory.newNormalizedName("s");
-    public final static byte[] SINGLE_COLUMN = SINGLE_COLUMN_NAME.getBytes();
-    public final static byte[] SINGLE_COLUMN_FAMILY = SINGLE_COLUMN_FAMILY_NAME.getBytes();
-
     public static final long AGG_TIMESTAMP = HConstants.LATEST_TIMESTAMP;
     /**
      * Key used for a single row aggregation where there is no group by
      */
     public final static byte[] UNGROUPED_AGG_ROW_KEY = Bytes.toBytes("a");
-    public final static PName AGG_COLUMN_NAME = SINGLE_COLUMN_NAME;
-    public final static PName AGG_COLUMN_FAMILY_NAME = SINGLE_COLUMN_FAMILY_NAME;
-
-    public static final byte[] ARRAY_VALUE_COLUMN_FAMILY = Bytes.toBytes("a");
-    // Use empty byte array for column qualifier so as not to accidentally conflict with any other columns
-    public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = ByteUtil.EMPTY_BYTE_ARRAY;
+    
+    /** BEGIN Set of reserved column qualifiers **/
+    
+    public static final String RESERVED_COLUMN_FAMILY = "_r";
+    public static final byte[] RESERVED_COLUMN_FAMILY_BYTES = Bytes.toBytes(RESERVED_COLUMN_FAMILY);
+    
+    public static final byte[] VALUE_COLUMN_FAMILY = RESERVED_COLUMN_FAMILY_BYTES;
+    public static final byte[] VALUE_COLUMN_QUALIFIER = getEncodedColumnQualifier(1);
+    
+    public static final byte[] ARRAY_VALUE_COLUMN_FAMILY = RESERVED_COLUMN_FAMILY_BYTES;
+    public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = getEncodedColumnQualifier(2);
+    
+    public final static PName SINGLE_COLUMN_NAME = PNameFactory.newNormalizedName("s");
+    public final static PName SINGLE_COLUMN_FAMILY_NAME = PNameFactory.newNormalizedName("s");
+    public final static byte[] SINGLE_COLUMN = getEncodedColumnQualifier(3);
+    public final static byte[] SINGLE_COLUMN_FAMILY = RESERVED_COLUMN_FAMILY_BYTES;
 
+    /** END Set of reserved column qualifiers **/
+    
     public static final byte[] TRUE = new byte[] {1};
     
     /**
@@ -192,11 +204,18 @@ public interface QueryConstants {
     public static final byte[] EMPTY_COLUMN_BYTES = Bytes.toBytes(EMPTY_COLUMN_NAME);
     public static final ImmutableBytesPtr EMPTY_COLUMN_BYTES_PTR = new ImmutableBytesPtr(
             EMPTY_COLUMN_BYTES);
+    public static final Integer ENCODED_EMPTY_COLUMN_NAME = 0;
+    public static final byte[] ENCODED_EMPTY_COLUMN_BYTES = getEncodedColumnQualifier(ENCODED_EMPTY_COLUMN_NAME);
+    public static final ImmutableBytesPtr ENCODED_EMPTY_COLUMN_BYTES_PTR = new ImmutableBytesPtr(
+            ENCODED_EMPTY_COLUMN_BYTES);
     public final static String EMPTY_COLUMN_VALUE = "x";
     public final static byte[] EMPTY_COLUMN_VALUE_BYTES = Bytes.toBytes(EMPTY_COLUMN_VALUE);
     public static final ImmutableBytesPtr EMPTY_COLUMN_VALUE_BYTES_PTR = new ImmutableBytesPtr(
             EMPTY_COLUMN_VALUE_BYTES);
-
+    public static final String ENCODED_EMPTY_COLUMN_VALUE = EMPTY_COLUMN_VALUE;
+    public final static byte[] ENCODED_EMPTY_COLUMN_VALUE_BYTES = Bytes.toBytes(EMPTY_COLUMN_VALUE);
+    public static final ImmutableBytesPtr ENCODED_EMPTY_COLUMN_VALUE_BYTES_PTR = new ImmutableBytesPtr(
+            ENCODED_EMPTY_COLUMN_VALUE_BYTES);
     public static final String DEFAULT_COLUMN_FAMILY = "0";
     public static final byte[] DEFAULT_COLUMN_FAMILY_BYTES = Bytes.toBytes(DEFAULT_COLUMN_FAMILY);
     public static final ImmutableBytesPtr DEFAULT_COLUMN_FAMILY_BYTES_PTR = new ImmutableBytesPtr(
@@ -222,6 +241,12 @@ public interface QueryConstants {
     public static final int NANOS_IN_SECOND = BigDecimal.valueOf(Math.pow(10, 9)).intValue();
     public static final int DIVERGED_VIEW_BASE_COLUMN_COUNT = -100;
     public static final int BASE_TABLE_BASE_COLUMN_COUNT = -1;
+    
+    /**
+     * We mark counter values 0 to 10 as reserved. Value 0 is used by {@link #ENCODED_EMPTY_COLUMN_NAME}. Values 1-10
+     * are reserved for special column qualifiers returned by Phoenix co-processors.
+     */
+    public static final int ENCODED_CQ_COUNTER_INITIAL_VALUE = 11;
     public static final String CREATE_TABLE_METADATA =
             // Do not use IF NOT EXISTS as we sometimes catch the TableAlreadyExists
             // exception and add columns to the SYSTEM.TABLE dynamically.
@@ -289,6 +314,9 @@ public interface QueryConstants {
             AUTO_PARTITION_SEQ + " VARCHAR," +
             APPEND_ONLY_SCHEMA + " BOOLEAN," +
             GUIDE_POSTS_WIDTH + " BIGINT," +
+            ENCODED_COLUMN_QUALIFIER + " UNSIGNED_INT," +
+            STORAGE_SCHEME + " TINYINT, " + 
+            COLUMN_QUALIFIER_COUNTER + " INTEGER, " +
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
             + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
             HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
@@ -363,5 +391,5 @@ public interface QueryConstants {
     public static final byte[] OFFSET_FAMILY = "f_offset".getBytes();
     public static final byte[] OFFSET_COLUMN = "c_offset".getBytes();
     public static final String LAST_SCAN = "LAST_SCAN";
-
+    public static final byte[] UPGRADE_MUTEX = "UPGRADE_MUTEX".getBytes();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
index 8c07ca6..3b4d9cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServicesOptions.java
@@ -243,7 +243,6 @@ public class QueryServicesOptions {
     public static final long DEFAULT_QUERY_SERVER_UGI_CACHE_MAX_SIZE = 1000L;
     public static final int DEFAULT_QUERY_SERVER_UGI_CACHE_INITIAL_SIZE = 100;
     public static final int DEFAULT_QUERY_SERVER_UGI_CACHE_CONCURRENCY = 10;
-
     public static final boolean DEFAULT_RENEW_LEASE_ENABLED = true;
     public static final int DEFAULT_RUN_RENEW_LEASE_FREQUENCY_INTERVAL_MILLISECONDS =
             DEFAULT_HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD / 2;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
index 1d772b4..88e7d77 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
@@ -17,10 +17,17 @@
  */
 package org.apache.phoenix.schema;
 
+import static org.apache.phoenix.util.EncodedColumnsUtil.usesEncodedColumnNames;
+
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.http.annotation.Immutable;
 import org.apache.phoenix.compile.ExpressionCompiler;
 import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
@@ -30,14 +37,12 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.schema.PTable.StorageScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.util.Arrays;
-
 
 /**
  * 
@@ -59,7 +64,7 @@ public class ColumnRef {
     }
 
     public ColumnRef(TableRef tableRef, String familyName, String columnName) throws MetaDataEntityNotFoundException {
-        this(tableRef, tableRef.getTable().getColumnFamily(familyName).getColumn(columnName).getPosition());
+        this(tableRef, tableRef.getTable().getColumnFamily(familyName).getPColumnForColumnName(columnName).getPosition());
     }
 
     public ColumnRef(TableRef tableRef, int columnPosition) {
@@ -123,7 +128,7 @@ public class ColumnRef {
         	return new ProjectedColumnExpression(column, table, displayName);
         }
 
-        Expression expression = new KeyValueColumnExpression(column, displayName);
+        Expression expression = new KeyValueColumnExpression(column, displayName, usesEncodedColumnNames(table));
 
         if (column.getExpressionStr() != null) {
             String url = PhoenixRuntime.JDBC_PROTOCOL
@@ -141,7 +146,10 @@ public class ColumnRef {
             }
         }
        
-        return expression;
+        if (table.getStorageScheme() == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+            return new ArrayColumnExpression(column, displayName, EncodedColumnsUtil.usesEncodedColumnNames(table.getStorageScheme()));
+        }
+        return new KeyValueColumnExpression(column, displayName, EncodedColumnsUtil.usesEncodedColumnNames(table.getStorageScheme()));
     }
 
     public ColumnRef cloneAtTimestamp(long timestamp) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
index 62d2e3f..cd9c2c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateColumn.java
@@ -100,4 +100,8 @@ public class DelegateColumn extends DelegateDatum implements PColumn {
     public boolean equals(Object o) {
 	    return getDelegate().equals(o);
 	}
+    @Override
+    public Integer getEncodedColumnQualifier() {
+        return getDelegate().getEncodedColumnQualifier();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 7d39dfe..b09fc5f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
@@ -96,8 +96,8 @@ public class DelegateTable implements PTable {
     }
 
     @Override
-    public PColumn getColumn(String name) throws ColumnNotFoundException, AmbiguousColumnException {
-        return delegate.getColumn(name);
+    public PColumn getPColumnForColumnName(String name) throws ColumnNotFoundException, AmbiguousColumnException {
+        return delegate.getPColumnForColumnName(name);
     }
 
     @Override
@@ -290,4 +290,19 @@ public class DelegateTable implements PTable {
     public boolean equals(Object obj) {
         return delegate.equals(obj);
     }
+    
+    @Override
+    public StorageScheme getStorageScheme() {
+        return delegate.getStorageScheme();
+    }
+
+    @Override
+    public PColumn getPColumnForColumnQualifier(byte[] cf, byte[] cq) throws ColumnNotFoundException, AmbiguousColumnException {
+        return delegate.getPColumnForColumnQualifier(cf, cq);
+    }
+
+    @Override
+    public EncodedCQCounter getEncodedCQCounter() {
+        return delegate.getEncodedCQCounter();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index ecd5f7a..f49dee7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -34,6 +34,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF;
 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.COLUMN_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
@@ -41,6 +42,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODED_COLUMN_QUALIFIER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
@@ -68,6 +70,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORAGE_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
@@ -84,9 +87,11 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RUN_UPDATE_STATS_ASYNC;
+import static org.apache.phoenix.schema.PTable.EncodedCQCounter.NULL_COUNTER;
 import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
 import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.schema.PTableType.VIEW;
@@ -147,7 +152,6 @@ import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
-import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -186,8 +190,10 @@ import org.apache.phoenix.query.ConnectionQueryServices.Feature;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.schema.PTable.EncodedCQCounter;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
 import org.apache.phoenix.schema.types.PDataType;
@@ -199,6 +205,7 @@ import org.apache.phoenix.schema.types.PUnsignedLong;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.LogUtil;
 import org.apache.phoenix.util.MetaDataUtil;
@@ -214,6 +221,7 @@ import org.apache.tephra.TxConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
@@ -258,8 +266,9 @@ public class MetaDataClient {
                     IS_NAMESPACE_MAPPED + "," +
                     AUTO_PARTITION_SEQ +  "," +
                     APPEND_ONLY_SCHEMA + "," +
-                    GUIDE_POSTS_WIDTH + 
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    GUIDE_POSTS_WIDTH + "," +
+                    STORAGE_SCHEME +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -274,6 +283,7 @@ public class MetaDataClient {
                     TABLE_SEQ_NUM +","+ // this is actually set to the parent table's sequence number
                     TABLE_TYPE +
                     ") VALUES (?, ?, ?, ?, ?, ?, ?)";
+    
     private static final String CREATE_VIEW_LINK =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
@@ -283,7 +293,17 @@ public class MetaDataClient {
                     LINK_TYPE + "," +
                     PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
                     ") VALUES (?, ?, ?, ?, ?, ?)";
-    private static final String INCREMENT_SEQ_NUM =
+    
+    public static final String UPDATE_ENCODED_COLUMN_COUNTER = 
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+            TENANT_ID + ", " + 
+            TABLE_SCHEM + "," +
+            TABLE_NAME + "," +
+            COLUMN_FAMILY + "," +
+            COLUMN_QUALIFIER_COUNTER + 
+            ") VALUES (?, ?, ?, ?, ?)";
+
+    public static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
@@ -336,8 +356,9 @@ public class MetaDataClient {
                     PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
                     KEY_SEQ + "," +
                     COLUMN_DEF + "," +
+                    ENCODED_COLUMN_QUALIFIER + ", " +
                     IS_ROW_TIMESTAMP +
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
@@ -357,8 +378,9 @@ public class MetaDataClient {
                     IS_VIEW_REFERENCED + "," +
                     PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
                     KEY_SEQ + "," +
-                    COLUMN_DEF +
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    COLUMN_DEF + "," +
+                    ENCODED_COLUMN_QUALIFIER +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
     private static final String UPDATE_COLUMN_POSITION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
                     TENANT_ID + "," +
@@ -703,22 +725,22 @@ public class MetaDataClient {
             // since view columns may be removed.
             IndexMaintainer indexMaintainer = index.getIndexMaintainer(parentTable, connection);
             // Check that the columns required for the index pk are present in the view
-            Set<ColumnReference> indexColRefs = indexMaintainer.getIndexedColumns();
-            for (ColumnReference colRef : indexColRefs) {
+            Set<Pair<String, String>> indexedColInfos = indexMaintainer.getIndexedColumnInfo();
+            for (Pair<String, String> colInfo : indexedColInfos) {
                 try {
-                    byte[] cf= colRef.getFamily();
-                    byte[] cq= colRef.getQualifier();
-                    if (cf!=null) {
-                        view.getColumnFamily(cf).getColumn(cq);
-                    }
-                    else {
-                        view.getColumn( Bytes.toString(cq));
+                    String colFamily = colInfo.getFirst();
+                    String colName = colInfo.getSecond();
+                    if (colFamily == null) {
+                        view.getPColumnForColumnName(colName);
+                    } else {
+                        view.getColumnFamily(colFamily).getPColumnForColumnName(colName);
                     }
-                } catch (ColumnNotFoundException e) { // Ignore this index and continue with others
+                } catch (ColumnNotFoundException e) {
                     containsAllReqdCols = false;
                     break;
                 }
             }
+            
             // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
             // all exist in the index on the parent table.
             for (PColumn col : view.getColumns()) {
@@ -728,17 +750,17 @@ public class MetaDataClient {
                         // but the WHERE clause for the view statement (which is added to the index below)
                         // would fail to compile.
                         String indexColumnName = IndexUtil.getIndexColumnName(col);
-                        index.getColumn(indexColumnName);
+                        index.getPColumnForColumnName(indexColumnName);
                     } catch (ColumnNotFoundException e1) {
                         PColumn indexCol = null;
                         try {
                             String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
-                            String cq = col.getName().getString();
-                            if (cf!=null) {
-                                indexCol = parentTable.getColumnFamily(cf).getColumn(cq);
+                            String colName = col.getName().getString();
+                            if (cf != null) {
+                                indexCol = parentTable.getColumnFamily(cf).getPColumnForColumnName(colName);
                             }
                             else {
-                                indexCol = parentTable.getColumn(cq);
+                                indexCol = parentTable.getPColumnForColumnName(colName);
                             }
                         } catch (ColumnNotFoundException e2) { // Ignore this index and continue with others
                             containsAllReqdCols = false;
@@ -805,8 +827,13 @@ public class MetaDataClient {
         } else {
             colUpsert.setString(18, column.getExpressionStr());
         }
-        if (colUpsert.getParameterMetaData().getParameterCount() > 18) {
-            colUpsert.setBoolean(19, column.isRowTimestamp());
+        if (column.getEncodedColumnQualifier() == null) {
+            colUpsert.setNull(19, Types.INTEGER);
+        } else {
+            colUpsert.setInt(19, column.getEncodedColumnQualifier());
+        }
+        if (colUpsert.getParameterMetaData().getParameterCount() > 19) {
+            colUpsert.setBoolean(20, column.isRowTimestamp());
         }
         colUpsert.execute();
     }
@@ -825,7 +852,7 @@ public class MetaDataClient {
         argUpsert.execute();
     }
 
-    private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint, String defaultColumnFamily, boolean addingToPK) throws SQLException {
+    private PColumn newColumn(int position, ColumnDef def, PrimaryKeyConstraint pkConstraint, String defaultColumnFamily, boolean addingToPK, Integer encodedColumnQualifier) throws SQLException {
         try {
             ColumnName columnDefName = def.getColumnDefName();
             SortOrder sortOrder = def.getSortOrder();
@@ -873,15 +900,14 @@ public class MetaDataClient {
                 }
                 isNull = false;
             }
-
             PColumn column = new PColumnImpl(PNameFactory.newName(columnName), familyName, def.getDataType(),
-                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp, false);
+                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp, false, isPK ? null : encodedColumnQualifier);
             return column;
         } catch (IllegalArgumentException e) { // Based on precondition check in constructor
             throw new SQLException(e);
         }
     }
-
+    
     public MutationState createTable(CreateTableStatement statement, byte[][] splits, PTable parent, String viewStatement, ViewType viewType, byte[][] viewColumnConstants, BitSet isViewColumnReferenced) throws SQLException {
         TableName tableName = statement.getTableName();
         Map<String,Object> tableProps = Maps.newHashMapWithExpectedSize(statement.getProps().size());
@@ -1776,7 +1802,6 @@ public class MetaDataClient {
             }
             String autoPartitionSeq = (String) TableProperty.AUTO_PARTITION_SEQ.getValue(tableProps);
             Long guidePostsWidth = (Long) TableProperty.GUIDE_POSTS_WIDTH.getValue(tableProps);
-
             Boolean storeNullsProp = (Boolean) TableProperty.STORE_NULLS.getValue(tableProps);
             if (storeNullsProp == null) {
                 if (parent == null) {
@@ -1798,7 +1823,7 @@ public class MetaDataClient {
                 if (transactionalProp == null) {
                     transactional = connection.getQueryServices().getProps().getBoolean(
                                     QueryServices.DEFAULT_TABLE_ISTRANSACTIONAL_ATTRIB,
-                                    QueryServicesOptions.DEFAULT_TRANSACTIONAL);
+                                    QueryServicesOptions.DEFAULT_TABLE_ISTRANSACTIONAL);
                 } else {
                     transactional = transactionalProp;
                 }
@@ -1813,7 +1838,7 @@ public class MetaDataClient {
                 .build().buildException();
             }
             // can't create a transactional table if it has a row timestamp column
-            if (pkConstraint.getNumColumnsWithRowTimestamp()>0 && transactional) {
+            if (pkConstraint.getNumColumnsWithRowTimestamp() > 0 && transactional) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP)
                 .setSchemaName(schemaName).setTableName(tableName)
                 .build().buildException();
@@ -1944,7 +1969,7 @@ public class MetaDataClient {
                 columns = new LinkedHashMap<PColumn,PColumn>(colDefs.size());
                 pkColumns = newLinkedHashSetWithExpectedSize(colDefs.size() + 1); // in case salted
             }
-
+            
             // Don't add link for mapped view, as it just points back to itself and causes the drop to
             // fail because it looks like there's always a view associated with it.
             if (!physicalNames.isEmpty()) {
@@ -1988,7 +2013,72 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
-
+            StorageScheme storageScheme = StorageScheme.NON_ENCODED_COLUMN_NAMES;
+            EncodedCQCounter cqCounter = NULL_COUNTER;
+            PTable viewPhysicalTable = null;
+            if (SchemaUtil.isSystemTable(Bytes.toBytes(SchemaUtil.getTableName(schemaName, tableName)))) {
+                // System tables have hard-coded column qualifiers. So we can't use column encoding for them.
+                storageScheme = StorageScheme.NON_ENCODED_COLUMN_NAMES;
+            } else if (tableType == PTableType.VIEW) {
+                /*
+                 * We can't control what column qualifiers are used in HTable mapped to Phoenix views. So we are not
+                 * able to encode column names.
+                 */  
+                if (viewType == MAPPED) {
+                    storageScheme = StorageScheme.NON_ENCODED_COLUMN_NAMES;
+                } else {
+                    /*
+                     * For regular phoenix views, use the storage scheme of the physical table since they all share the
+                     * the same HTable. Views always use the base table's column qualifier counter for doling out
+                     * encoded column qualifier.
+                     */
+                    viewPhysicalTable = PhoenixRuntime.getTable(connection, physicalNames.get(0).getString());
+                    storageScheme = viewPhysicalTable.getStorageScheme();
+					if (EncodedColumnsUtil.usesEncodedColumnNames(viewPhysicalTable)) {
+                        cqCounter  = viewPhysicalTable.getEncodedCQCounter();
+                    }
+                }
+            } else {
+                /*
+                 * New indexes on existing tables can have encoded column names. But unfortunately, due to backward
+                 * compatibility reasons, we aren't able to change IndexMaintainer and the state that is serialized in
+                 * it. Because of this we are forced to have the indexes inherit the storage scheme of the parent data
+                 * tables. Otherwise, we always attempt to create tables with encoded column names. 
+                 * 
+                 * Also of note is the case with shared indexes i.e. local indexes and view indexes. In these cases, 
+                 * column qualifiers for covered columns don't have to be unique because rows of the logical indexes are 
+                 * partitioned by the virtue of indexId present in the row key. As such, different shared indexes can use
+                 * potentially overlapping column qualifiers.
+                 * 
+                 * If the hbase table already exists, then possibly encoded or non-encoded column qualifiers already exist. 
+                 * In this case we pursue ahead with non-encoded column qualifier scheme. If the phoenix table already exists 
+                 * then we rely on the PTable, with appropriate storage scheme, returned in the MetadataMutationResult to be updated 
+                 * in the client cache. If the phoenix table already doesn't exist then the non-encoded column qualifier scheme works
+                 * because we cannot control the column qualifiers that were used when populating the hbase table.
+                 */
+                byte[] tableNameBytes = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
+                boolean tableExists = true;
+                try {
+                    connection.getQueryServices().getTableDescriptor(tableNameBytes);
+                } catch (org.apache.phoenix.schema.TableNotFoundException e) {
+                    tableExists = false;
+                }
+
+                if (parent != null) {
+                    storageScheme = parent.getStorageScheme();
+                } else if (tableExists) {
+                    storageScheme = StorageScheme.NON_ENCODED_COLUMN_NAMES;
+                } else if (isImmutableRows) {
+                    storageScheme = StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL;
+                    // since we are storing all columns of a column family in a single key value we can't use deletes to store nulls
+                    storeNulls = true;
+                } else {
+                    storageScheme = StorageScheme.ENCODED_COLUMN_NAMES;
+                }
+                cqCounter = storageScheme != StorageScheme.NON_ENCODED_COLUMN_NAMES ? new EncodedCQCounter() : NULL_COUNTER;
+            }
+            
+            Map<String, Integer> changedCqCounters = new HashMap<>(colDefs.size());
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2007,7 +2097,24 @@ public class MetaDataClient {
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                     }
                 }
-                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false);
+                ColumnName columnDefName = colDef.getColumnDefName();
+                String colDefFamily = columnDefName.getFamilyName();
+                boolean isPkColumn = isPkColumn(pkConstraint, colDef, columnDefName);
+                String cqCounterFamily = null;
+                if (!isPkColumn) {
+                    if (storageScheme == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+                        // For this scheme we track column qualifier counters at the column family level.
+                        cqCounterFamily = colDefFamily != null ? colDefFamily : (defaultFamilyName != null ? defaultFamilyName : DEFAULT_COLUMN_FAMILY);
+                    } else {
+                        // For other schemes, column qualifier counters are tracked using the default column family.
+                        cqCounterFamily = defaultFamilyName != null ? defaultFamilyName : DEFAULT_COLUMN_FAMILY;
+                    }
+                }
+                Integer encodedCQ =  isPkColumn ? null : cqCounter.getNextQualifier(cqCounterFamily);   
+                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false, encodedCQ);
+                if (cqCounter.increment(cqCounterFamily)) {
+                    changedCqCounters.put(cqCounterFamily, cqCounter.getNextQualifier(cqCounterFamily));
+                }
                 if (SchemaUtil.isPKColumn(column)) {
                     // TODO: remove this constraint?
                     if (pkColumnsIterator.hasNext() && !column.getName().getString().equals(pkColumnsIterator.next().getFirst().getColumnName())) {
@@ -2042,6 +2149,9 @@ public class MetaDataClient {
                         column.getFamilyName());
                 }
             }
+            
+            
+            
             // We need a PK definition for a TABLE or mapped VIEW
             if (!isPK && pkColumnsNames.isEmpty() && tableType != PTableType.VIEW && viewType != ViewType.MAPPED) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
@@ -2123,15 +2233,52 @@ public class MetaDataClient {
             if (SchemaUtil.isMetaTable(schemaName,tableName)) {
                 // TODO: what about stats for system catalog?
                 PName newSchemaName = PNameFactory.newName(schemaName);
+                // Column names and qualifiers and hardcoded for system tables.
                 PTable table = PTableImpl.makePTable(tenantId,newSchemaName, PNameFactory.newName(tableName), tableType,
                         null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                         PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns.values(), null, null,
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
+            
+            // Update column qualifier counters
+            if (EncodedColumnsUtil.usesEncodedColumnNames(storageScheme)) {
+                // Store the encoded column counter for phoenix entities that have their own hbase
+                // tables i.e. base tables and indexes.
+                String schemaNameToUse = tableType == VIEW ? viewPhysicalTable.getSchemaName().getString() : schemaName;
+                String tableNameToUse = tableType == VIEW ? viewPhysicalTable.getTableName().getString() : tableName;
+                boolean sharedIndex = tableType == PTableType.INDEX && (indexType == IndexType.LOCAL || parent.getType() == PTableType.VIEW);
+                // For local indexes and indexes on views, pass on the the tenant id since all their meta-data rows have
+                // tenant ids in there.
+                String tenantIdToUse = connection.getTenantId() != null && sharedIndex ? connection.getTenantId().getString() : null;
+                // When a view adds its own columns, then we need to increase the sequence number of the base table
+                // too since we want clients to get the latest PTable of the base table.
+                for (Entry<String, Integer> entry : changedCqCounters.entrySet()) {
+                    try (PreparedStatement linkStatement = connection.prepareStatement(UPDATE_ENCODED_COLUMN_COUNTER)) {
+                        linkStatement.setString(1, tenantIdToUse);
+                        linkStatement.setString(2, schemaNameToUse);
+                        linkStatement.setString(3, tableNameToUse);
+                        linkStatement.setString(4, entry.getKey());
+                        linkStatement.setInt(5, entry.getValue());
+                        linkStatement.execute();
+                    }
+                }
+                if (tableType == VIEW && !changedCqCounters.isEmpty()) { //TODO: samarth think about shared indexes
+                    PreparedStatement incrementStatement = connection.prepareStatement(INCREMENT_SEQ_NUM);
+                    incrementStatement.setString(1, null);
+                    incrementStatement.setString(2, viewPhysicalTable.getSchemaName().getString());
+                    incrementStatement.setString(3, viewPhysicalTable.getTableName().getString());
+                    incrementStatement.setLong(4, viewPhysicalTable.getSequenceNumber() + 1);
+                    incrementStatement.execute();
+                }
+                if (connection.getMutationState().toMutations(timestamp).hasNext()) {
+                    tableMetaData.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
+                    connection.rollback();
+                }
+            }
 
             short nextKeySeq = 0;
 
@@ -2157,14 +2304,14 @@ public class MetaDataClient {
                                 return true;
                             }
                         });
-                    }
-                    else if (isViewColumnReferenced != null) {
+                    } else if (isViewColumnReferenced != null) {
                         if (viewColumnConstants != null && columnPosition < viewColumnConstants.length) {
                             entry.setValue(column = new DelegateColumn(column) {
                                 @Override
                                 public byte[] getViewConstant() {
                                     return viewColumnConstants[columnPosition];
                                 }
+                                
                                 @Override
                                 public boolean isViewReferenced() {
                                     return isViewColumnReferenced.get(columnPosition);
@@ -2247,6 +2394,7 @@ public class MetaDataClient {
             } else {
                 tableUpsert.setLong(25, guidePostsWidth);
             }
+            tableUpsert.setByte(26, storageScheme.getSerializedValue()); //TODO: samarth should there be a null check here?
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2267,7 +2415,7 @@ public class MetaDataClient {
              * 3) parent table header row
              */
             Collections.reverse(tableMetaData);
-
+            
 			if (indexType != IndexType.LOCAL) {
                 splits = SchemaUtil.processSplits(splits, pkColumns, saltBucketNum, connection.getQueryServices().getProps().getBoolean(
                         QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, QueryServicesOptions.DEFAULT_FORCE_ROW_KEY_ORDER));
@@ -2340,12 +2488,18 @@ public class MetaDataClient {
                     }
                 }
                 PName newSchemaName = PNameFactory.newName(schemaName);
+                /*
+                 * It doesn't hurt for the PTable of views to have the cqCounter. However, views always rely on the
+                 * parent table's counter to dole out encoded column qualifiers. So setting the counter as NULL_COUNTER
+                 * for extra safety.
+                 */
+                EncodedCQCounter cqCounterToBe = tableType == PTableType.VIEW ? NULL_COUNTER : cqCounter;
                 PTable table =  PTableImpl.makePTable(
                         tenantId, newSchemaName, PNameFactory.newName(tableName), tableType, indexState, timestamp!=null ? timestamp : result.getMutationTime(),
                         PTable.INITIAL_SEQ_NUM, pkName == null ? null : PNameFactory.newName(pkName), saltBucketNum, columns.values(),
                         parent == null ? null : parent.getSchemaName(), parent == null ? null : parent.getTableName(), Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : PNameFactory.newName(defaultFamilyName), viewStatement, Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
+                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, cqCounterToBe);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -2355,6 +2509,10 @@ public class MetaDataClient {
         }
     }
 
+    private static boolean isPkColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef, ColumnName columnDefName) {
+        return colDef.isPK() || (pkConstraint != null && pkConstraint.getColumnWithSortOrder(columnDefName) != null);
+    }
+    
     /**
      * A table can be a parent table to tenant-specific tables if all of the following conditions are true:
      * <p>
@@ -2749,7 +2907,6 @@ public class MetaDataClient {
             PName tenantId = connection.getTenantId();
             String schemaName = table.getSchemaName().getString();
             String tableName = table.getTableName().getString();
-
             Boolean isImmutableRowsProp = null;
             Boolean multiTenantProp = null;
             Boolean disableWALProp = null;
@@ -2769,7 +2926,7 @@ public class MetaDataClient {
                     if (familyName!=null) {
                         try {
                             PColumnFamily columnFamily = table.getColumnFamily(familyName);
-                            columnFamily.getColumn(columnName);
+                            columnFamily.getPColumnForColumnName(columnName);
                             if (!ifNotExists) {
                                 throw new ColumnAlreadyExistsException(schemaName, tableName, columnName);
                             }
@@ -2780,7 +2937,7 @@ public class MetaDataClient {
                     }
                     else {
                         try {
-                            table.getColumn(columnName);
+                            table.getPColumnForColumnName(columnName);
                             if (!ifNotExists) {
                                 throw new ColumnAlreadyExistsException(schemaName, tableName, columnName);
                             }
@@ -2833,7 +2990,8 @@ public class MetaDataClient {
                 ColumnResolver resolver = FromCompiler.getResolver(namedTableNode, connection);
                 table = resolver.getTables().get(0).getTable();
                 int nIndexes = table.getIndexes().size();
-                int nNewColumns = columnDefs.size();
+                int numCols = columnDefs.size();
+                int nNewColumns = numCols;
                 List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize((1 + nNewColumns) * (nIndexes + 1));
                 List<Mutation> columnMetaData = Lists.newArrayListWithExpectedSize(nNewColumns * (nIndexes + 1));
                 if (logger.isDebugEnabled()) {
@@ -2926,11 +3084,15 @@ public class MetaDataClient {
                 Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || nonTxToTx);
 
                 int numPkColumnsAdded = 0;
-                List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnDefs.size());
+                List<PColumn> columns = Lists.newArrayListWithExpectedSize(numCols);
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
                 Set<String> families = new LinkedHashSet<>();
-                if (columnDefs.size() > 0 ) {
+                PTable tableForCQCounters = tableType == PTableType.VIEW ? PhoenixRuntime.getTable(connection, table.getPhysicalName().getString()) : table;;
+                EncodedCQCounter cqCounterToUse = tableForCQCounters.getEncodedCQCounter();
+                Map<String, Integer> changedCqCounters = new HashMap<>(numCols);
+                if (numCols > 0 ) {
                     StatementContext context = new StatementContext(new PhoenixStatement(connection), resolver);
+                    //TODO: samarth should these be guarded by storage scheme check. Better to have the map always available. immutable empty for views and non encoded.
                     try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_ALTER_TABLE)) {
                         short nextKeySeq = SchemaUtil.getMaxKeySeq(table);
                         for( ColumnDef colDef : columnDefs) {
@@ -2953,11 +3115,32 @@ public class MetaDataClient {
                             if (!colDef.validateDefault(context, null)) {
                                 colDef = new ColumnDef(colDef, null); // Remove DEFAULT as it's not necessary
                             }
-                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
+                            Integer encodedCQ = null;
+                            if (!colDef.isPK()) {
+                                String colDefFamily = colDef.getColumnDefName().getFamilyName();
+                                String familyName = null;
+                                StorageScheme storageScheme = table.getStorageScheme();
+                                String defaultColumnFamily = tableForCQCounters.getDefaultFamilyName() != null && !Strings.isNullOrEmpty(tableForCQCounters.getDefaultFamilyName().getString()) ? 
+                                        tableForCQCounters.getDefaultFamilyName().getString() : DEFAULT_COLUMN_FAMILY;
+                                    if (table.getType() == PTableType.INDEX && table.getIndexType() == IndexType.LOCAL) {
+                                        defaultColumnFamily = QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX + defaultColumnFamily;
+                                    }
+                                if (storageScheme == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+                                    familyName = colDefFamily != null ? colDefFamily : defaultColumnFamily;
+                                } else {
+                                    familyName = defaultColumnFamily;
+                                }
+                                encodedCQ = cqCounterToUse.getNextQualifier(familyName);
+                                if (cqCounterToUse.increment(familyName)) {
+                                    changedCqCounters.put(familyName,
+                                        cqCounterToUse.getNextQualifier(familyName));
+                                }
+                            }
+                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true, encodedCQ);
                             columns.add(column);
                             String pkName = null;
                             Short keySeq = null;
-
+                            
                             // TODO: support setting properties on other families?
                             if (column.getFamilyName() == null) {
                                 ++numPkColumnsAdded;
@@ -2969,13 +3152,13 @@ public class MetaDataClient {
                             colFamiliesForPColumnsToBeAdded.add(column.getFamilyName() == null ? null : column.getFamilyName().getString());
                             addColumnMutation(schemaName, tableName, column, colUpsert, null, pkName, keySeq, table.getBucketNum() != null);
                         }
-
+                        
                         // Add any new PK columns to end of index PK
-                        if (numPkColumnsAdded>0) {
+                        if (numPkColumnsAdded > 0) {
                             // create PK column list that includes the newly created columns
                             List<PColumn> pkColumns = Lists.newArrayListWithExpectedSize(table.getPKColumns().size()+numPkColumnsAdded);
                             pkColumns.addAll(table.getPKColumns());
-                            for (int i=0; i<columnDefs.size(); ++i) {
+                            for (int i=0; i<numCols; ++i) {
                                 if (columnDefs.get(i).isPK()) {
                                     pkColumns.add(columns.get(i));
                                 }
@@ -2984,14 +3167,14 @@ public class MetaDataClient {
                             for (PTable index : table.getIndexes()) {
                                 short nextIndexKeySeq = SchemaUtil.getMaxKeySeq(index);
                                 int indexPosition = index.getColumns().size();
-                                for (int i=0; i<columnDefs.size(); ++i) {
+                                for (int i=0; i<numCols; ++i) {
                                     ColumnDef colDef = columnDefs.get(i);
                                     if (colDef.isPK()) {
                                         PDataType indexColDataType = IndexUtil.getIndexColumnDataType(colDef.isNull(), colDef.getDataType());
                                         ColumnName indexColName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, colDef.getColumnDefName().getColumnName()));
                                         Expression expression = new RowKeyColumnExpression(columns.get(i), new RowKeyValueAccessor(pkColumns, ++pkSlotPosition));
                                         ColumnDef indexColDef = FACTORY.columnDef(indexColName, indexColDataType.getSqlTypeName(), colDef.isNull(), colDef.getMaxLength(), colDef.getScale(), true, colDef.getSortOrder(), expression.toString(), colDef.isRowTimestamp());
-                                        PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null, true);
+                                        PColumn indexColumn = newColumn(indexPosition++, indexColDef, PrimaryKeyConstraint.EMPTY, null, true, null);
                                         addColumnMutation(schemaName, index.getTableName().getString(), indexColumn, colUpsert, index.getParentTableName().getString(), index.getPKName() == null ? null : index.getPKName().getString(), ++nextIndexKeySeq, index.getBucketNum() != null);
                                     }
                                 }
@@ -3027,10 +3210,10 @@ public class MetaDataClient {
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
+                
                 long seqNum = table.getSequenceNumber();
-                if (changingPhoenixTableProperty || columnDefs.size() > 0) {
-                    seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows,
-                            disableWAL, multiTenant, storeNulls, guidePostWidth);
+                if (changingPhoenixTableProperty || numCols > 0) { 
+                    seqNum = incrementTableSeqNum(table, tableType, numCols, isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls, guidePostWidth);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -3039,6 +3222,33 @@ public class MetaDataClient {
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
                 tableMetaData.addAll(columnMetaData);
+                boolean sharedIndex = tableType == PTableType.INDEX && (table.getIndexType() == IndexType.LOCAL || table.getViewIndexId() != null);
+                String tenantIdToUse = connection.getTenantId() != null && sharedIndex ? connection.getTenantId().getString() : null;
+                if (!changedCqCounters.isEmpty()) {
+                    PreparedStatement linkStatement;
+                        linkStatement = connection.prepareStatement(UPDATE_ENCODED_COLUMN_COUNTER);
+                        for (Entry<String, Integer> entry : changedCqCounters.entrySet()) {    
+                            linkStatement.setString(1, tenantIdToUse);
+                            linkStatement.setString(2, tableForCQCounters.getSchemaName().getString());
+                            linkStatement.setString(3, tableForCQCounters.getTableName().getString());
+                            linkStatement.setString(4, entry.getKey());
+                            linkStatement.setInt(5, entry.getValue());
+                            linkStatement.execute();
+                        }
+
+                    // When a view adds its own columns, then we need to increase the sequence number of the base table
+                    // too since we want clients to get the latest PTable of the base table.
+                    if (tableType == VIEW) {
+                        PreparedStatement incrementStatement = connection.prepareStatement(INCREMENT_SEQ_NUM);
+                        incrementStatement.setString(1, null); //TODO: samarth verify that tenant id should be null here
+                        incrementStatement.setString(2, tableForCQCounters.getSchemaName().getString());
+                        incrementStatement.setString(3, tableForCQCounters.getTableName().getString());
+                        incrementStatement.setLong(4, tableForCQCounters.getSequenceNumber() + 1);
+                        incrementStatement.execute();
+                    }
+                    tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
+                    connection.rollback();
+                }
 
                 byte[] family = families.size() > 0 ? families.iterator().next().getBytes() : null;
 
@@ -3068,7 +3278,6 @@ public class MetaDataClient {
                         }
                         return new MutationState(0,connection);
                     }
-
                     // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
@@ -3089,6 +3298,7 @@ public class MetaDataClient {
                                                                         updateCacheFrequency == null ? table.getUpdateCacheFrequency() : updateCacheFrequency,
                                                                                 table.isNamespaceMapped(),
                                                                                 resolvedTimeStamp);
+                        table = connection.getTable(new PTableKey(connection.getTenantId(), table.getName().getString()));
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3265,23 +3475,23 @@ public class MetaDataClient {
                 Long timeStamp = table.isTransactional() ? tableRef.getTimeStamp() : null;
                 for (PTable index : table.getIndexes()) {
                     IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
-                    // get the columns required for the index pk
-                    Set<ColumnReference> indexColumns = indexMaintainer.getIndexedColumns();
-                    // get the covered columns
-                    Set<ColumnReference> coveredColumns = indexMaintainer.getCoveredColumns();
+                    // get the covered columns 
                     List<PColumn> indexColumnsToDrop = Lists.newArrayListWithExpectedSize(columnRefs.size());
+                    Set<Pair<String, String>> indexedColsInfo = indexMaintainer.getIndexedColumnInfo();
+                    Set<Pair<String, String>> coveredColsInfo = indexMaintainer.getCoveredColumnInfo();
                     for(PColumn columnToDrop : tableColumnsToDrop) {
-                        ColumnReference columnToDropRef = new ColumnReference(columnToDrop.getFamilyName().getBytes(), columnToDrop.getName().getBytes());
-                        // if the columns being dropped is indexed and the physical index table is not shared
-                        if (indexColumns.contains(columnToDropRef)) {
-                            if (index.getViewIndexId()==null)
+                        Pair<String, String> columnToDropInfo = new Pair<>(columnToDrop.getFamilyName().getString(), columnToDrop.getName().getString());
+                        boolean isColumnIndexed = indexedColsInfo.contains(columnToDropInfo);
+                        if (isColumnIndexed) {
+                            if (index.getViewIndexId() == null) { 
                                 indexesToDrop.add(new TableRef(index));
+                            }
                             connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, index.getName().getString()), index.getParentName() == null ? null : index.getParentName().getString(), index.getTimeStamp());
                             removedIndexTableOrColumn = true;
                         }
-                        else if (coveredColumns.contains(columnToDropRef)) {
+                        else if (coveredColsInfo.contains(columnToDropInfo)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
-                            PColumn indexColumn = index.getColumn(indexColumnName);
+                            PColumn indexColumn = index.getPColumnForColumnName(indexColumnName);
                             indexColumnsToDrop.add(indexColumn);
                             // add the index column to be dropped so that we actually delete the column values
                             columnsToDrop.add(new ColumnRef(new TableRef(index), indexColumn.getPosition()));
@@ -3371,13 +3581,15 @@ public class MetaDataClient {
                         // so we need to issue deletes markers for all the rows of the index
                         final List<TableRef> tableRefsToDrop = Lists.newArrayList();
                         Map<String, List<TableRef>> tenantIdTableRefMap = Maps.newHashMap();
-                        if (result.getSharedTablesToDelete()!=null) {
+                        if (result.getSharedTablesToDelete() != null) {
                             for (SharedTableState sharedTableState : result.getSharedTablesToDelete()) {
+                                //TODO: samarth I don't think we really care about storage scheme and cq counter at this point.
+                                //Probably worthy to change the constructor here to not expect the two arguments.
                                 PTableImpl viewIndexTable = new PTableImpl(sharedTableState.getTenantId(),
                                         sharedTableState.getSchemaName(), sharedTableState.getTableName(), ts,
                                         table.getColumnFamilies(), sharedTableState.getColumns(),
                                         sharedTableState.getPhysicalNames(), sharedTableState.getViewIndexId(),
-                                        table.isMultiTenant(), table.isNamespaceMapped());
+                                        table.isMultiTenant(), table.isNamespaceMapped(), table.getStorageScheme(), table.getEncodedCQCounter());
                                 TableRef indexTableRef = new TableRef(viewIndexTable);
                                 PName indexTableTenantId = sharedTableState.getTenantId();
                                 if (indexTableTenantId==null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
index 0f5fa44..978ded7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
@@ -27,7 +27,7 @@ package org.apache.phoenix.schema;
 public interface PColumn extends PDatum {
 
     /**
-     * @return the name of the column qualifier
+     * @return the name of the column
      */
     PName getName();
 
@@ -60,4 +60,6 @@ public interface PColumn extends PDatum {
     boolean isRowTimestamp();
     
     boolean isDynamic();
+    
+    Integer getEncodedColumnQualifier();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
index 24da14d..c4c383e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamily.java
@@ -39,16 +39,22 @@ public interface PColumnFamily {
     Collection<PColumn> getColumns();
     
     /**
-     * @return The PColumn for the specified column qualifier.
+     * @return The PColumn for the specified column name.
      * @throws ColumnNotFoundException if the column cannot be found
      */
-    PColumn getColumn(byte[] qualifier) throws ColumnNotFoundException;
+    PColumn getPColumnForColumnNameBytes(byte[] columnNameBytes) throws ColumnNotFoundException;
     
     /**
-     * @return The PColumn for the specified column qualifier.
+     * @return The PColumn for the specified column name.
      * @throws ColumnNotFoundException if the column cannot be found
      */
-    PColumn getColumn(String name) throws ColumnNotFoundException;
+    PColumn getPColumnForColumnName(String columnName) throws ColumnNotFoundException;
     
     int getEstimatedSize();
+    
+    /**
+     * @return The PColumn for the specified column qualifier.
+     * @throws ColumnNotFoundException if the column cannot be found
+     */
+    PColumn getPColumnForColumnQualifier(byte[] cq) throws ColumnNotFoundException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
index 2e29656..c175aa2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnFamilyImpl.java
@@ -21,6 +21,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.SizedUtil;
 
 import com.google.common.base.Preconditions;
@@ -31,8 +32,9 @@ import com.google.common.collect.ImmutableSortedMap;
 public class PColumnFamilyImpl implements PColumnFamily {
     private final PName name;
     private final List<PColumn> columns;
-    private final Map<String, PColumn> columnByString;
-    private final Map<byte[], PColumn> columnByBytes;
+    private final Map<String, PColumn> columnNamesByStrings;
+    private final Map<byte[], PColumn> columnNamesByBytes;
+    private final Map<byte[], PColumn> encodedColumnQualifersByBytes;
     private final int estimatedSize;
 
     @Override
@@ -40,22 +42,27 @@ public class PColumnFamilyImpl implements PColumnFamily {
         return estimatedSize;
     }
     
-    public PColumnFamilyImpl(PName name, List<PColumn> columns) {
+    public PColumnFamilyImpl(PName name, List<PColumn> columns, boolean useEncodedColumnNames) {
         Preconditions.checkNotNull(name);
         // Include guidePosts also in estimating the size
         long estimatedSize = SizedUtil.OBJECT_SIZE + SizedUtil.POINTER_SIZE * 5 + SizedUtil.INT_SIZE + name.getEstimatedSize() +
                 SizedUtil.sizeOfMap(columns.size()) * 2 + SizedUtil.sizeOfArrayList(columns.size());
         this.name = name;
         this.columns = ImmutableList.copyOf(columns);
-        ImmutableMap.Builder<String, PColumn> columnByStringBuilder = ImmutableMap.builder();
-        ImmutableSortedMap.Builder<byte[], PColumn> columnByBytesBuilder = ImmutableSortedMap.orderedBy(Bytes.BYTES_COMPARATOR);
+        ImmutableMap.Builder<String, PColumn> columnNamesByStringBuilder = ImmutableMap.builder();
+        ImmutableSortedMap.Builder<byte[], PColumn> columnNamesByBytesBuilder = ImmutableSortedMap.orderedBy(Bytes.BYTES_COMPARATOR);
+        ImmutableSortedMap.Builder<byte[], PColumn> encodedColumnQualifiersByBytesBuilder = ImmutableSortedMap.orderedBy(Bytes.BYTES_COMPARATOR);
         for (PColumn column : columns) {
             estimatedSize += column.getEstimatedSize();
-            columnByBytesBuilder.put(column.getName().getBytes(), column);
-            columnByStringBuilder.put(column.getName().getString(), column);
+            columnNamesByBytesBuilder.put(column.getName().getBytes(), column);
+            columnNamesByStringBuilder.put(column.getName().getString(), column);
+            if (useEncodedColumnNames && column.getEncodedColumnQualifier() != null) {
+                encodedColumnQualifiersByBytesBuilder.put(EncodedColumnsUtil.getEncodedColumnQualifier(column), column);
+            }
         }
-        this.columnByBytes = columnByBytesBuilder.build();
-        this.columnByString = columnByStringBuilder.build();
+        this.columnNamesByBytes = columnNamesByBytesBuilder.build();
+        this.columnNamesByStrings = columnNamesByStringBuilder.build();
+        this.encodedColumnQualifersByBytes =  encodedColumnQualifiersByBytesBuilder.build();
         this.estimatedSize = (int)estimatedSize;
     }
     
@@ -70,19 +77,32 @@ public class PColumnFamilyImpl implements PColumnFamily {
     }
 
     @Override
-    public PColumn getColumn(byte[] qualifier) throws ColumnNotFoundException  {
-        PColumn column = columnByBytes.get(qualifier);
+    public PColumn getPColumnForColumnNameBytes(byte[] columnNameBytes) throws ColumnNotFoundException  {
+        PColumn column = columnNamesByBytes.get(columnNameBytes);
         if (column == null) {
-            throw new ColumnNotFoundException(Bytes.toString(qualifier));
+            throw new ColumnNotFoundException(Bytes.toString(columnNameBytes));
         }
         return column;
     }
     
     @Override
-    public PColumn getColumn(String name) throws ColumnNotFoundException  {
-        PColumn column = columnByString.get(name);
+    public PColumn getPColumnForColumnName(String columnName) throws ColumnNotFoundException  {
+        PColumn column = columnNamesByStrings.get(columnName);
         if (column == null) {
-            throw new ColumnNotFoundException(name);
+            throw new ColumnNotFoundException(columnName);
+        }
+        return column;
+    }
+    
+    @Override
+    public PColumn getPColumnForColumnQualifier(byte[] cq) throws ColumnNotFoundException {
+        Preconditions.checkNotNull(cq);
+        PColumn column = encodedColumnQualifersByBytes.get(cq);
+        if (column == null) {
+            // For tables with non-encoded column names, column qualifiers are
+            // column name bytes. Also dynamic columns don't have encoded column
+            // qualifiers. So they could be found in the column name by bytes map.
+            return getPColumnForColumnNameBytes(cq);
         }
         return column;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/70e00a38/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index ca827d8..e6f8b71 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
@@ -40,6 +40,7 @@ public class PColumnImpl implements PColumn {
     private String expressionStr;
     private boolean isRowTimestamp;
     private boolean isDynamic;
+    private Integer columnQualifier;
     
     public PColumnImpl() {
     }
@@ -51,13 +52,13 @@ public class PColumnImpl implements PColumn {
                        Integer scale,
                        boolean nullable,
                        int position,
-                       SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic) {
-        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic);
+                       SortOrder sortOrder, Integer arrSize, byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic, Integer columnQualifier) {
+        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifier);
     }
 
     public PColumnImpl(PColumn column, int position) {
         this(column.getName(), column.getFamilyName(), column.getDataType(), column.getMaxLength(),
-                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic());
+                column.getScale(), column.isNullable(), position, column.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getEncodedColumnQualifier());
     }
 
     private void init(PName name,
@@ -69,7 +70,7 @@ public class PColumnImpl implements PColumn {
             int position,
             SortOrder sortOrder,
             Integer arrSize,
-            byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic) {
+            byte[] viewConstant, boolean isViewReferenced, String expressionStr, boolean isRowTimestamp, boolean isDynamic, Integer columnQualifier) {
     	Preconditions.checkNotNull(sortOrder);
         this.dataType = dataType;
         if (familyName == null) {
@@ -94,6 +95,7 @@ public class PColumnImpl implements PColumn {
         this.expressionStr = expressionStr;
         this.isRowTimestamp = isRowTimestamp;
         this.isDynamic = isDynamic;
+        this.columnQualifier = columnQualifier;
     }
 
     @Override
@@ -205,6 +207,11 @@ public class PColumnImpl implements PColumn {
     public boolean isDynamic() {
         return isDynamic;
     }
+    
+    @Override
+    public Integer getEncodedColumnQualifier() {
+        return columnQualifier;
+    }
 
     /**
      * Create a PColumn instance from PBed PColumn instance
@@ -251,8 +258,12 @@ public class PColumnImpl implements PColumn {
         if (column.hasIsDynamic()) {
         	isDynamic = column.getIsDynamic();
         }
+        Integer columnQualifier = null;
+        if (column.hasColumnQualifier()) {
+            columnQualifier = column.getColumnQualifier();
+        }
         return new PColumnImpl(columnName, familyName, dataType, maxLength, scale, nullable, position, sortOrder,
-                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic);
+                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifier);
     }
 
     public static PTableProtos.PColumn toProto(PColumn column) {
@@ -283,6 +294,9 @@ public class PColumnImpl implements PColumn {
             builder.setExpression(column.getExpressionStr());
         }
         builder.setIsRowTimestamp(column.isRowTimestamp());
+        if (column.getEncodedColumnQualifier() != null) {
+            builder.setColumnQualifier(column.getEncodedColumnQualifier());
+        }
         return builder.build();
     }
 }