http://git-wip-us.apache.org/repos/asf/phoenix/blob/450dbc59/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 1264e32..c9cefaf 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
@@ -35,6 +35,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;
@@ -42,6 +43,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.IMMUTABLE_ROWS;
 import static 
org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
@@ -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;
@@ -85,6 +88,7 @@ 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.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
+import static 
org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
 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;
@@ -188,8 +192,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.PTableStats;
 import org.apache.phoenix.schema.types.PDataType;
@@ -201,6 +207,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;
@@ -223,6 +230,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.common.primitives.Ints;
+import static org.apache.phoenix.schema.PTable.EncodedCQCounter.NULL_COUNTER;
 
 public class MetaDataClient {
     private static final Logger logger = 
LoggerFactory.getLogger(MetaDataClient.class);
@@ -261,9 +269,10 @@ public class MetaDataClient {
             UPDATE_CACHE_FREQUENCY + "," +
             IS_NAMESPACE_MAPPED + "," +
             AUTO_PARTITION_SEQ +  "," +
-            APPEND_ONLY_SCHEMA +
-            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, 
?, ?, ?, ?, ?, ?, ?)";
-
+            APPEND_ONLY_SCHEMA + "," +
+            STORAGE_SCHEME +
+            ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, 
?, ?, ?, ?, ?, ?, ?, ?)";
+            
     private static final String CREATE_SCHEMA = "UPSERT INTO " + 
SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
 
@@ -286,6 +295,14 @@ public class MetaDataClient {
             LINK_TYPE + "," +
             PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // 
Dynamic column for now to prevent schema change
             ") VALUES (?, ?, ?, ?, ?, ?)";
+    private 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 (?, ?, ?, ?, ?)";
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + 
SYSTEM_CATALOG_TABLE + "\"( " +
             TENANT_ID + "," +
@@ -339,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 + "," +
-        IS_ROW_TIMESTAMP + 
-        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+        ENCODED_COLUMN_QUALIFIER + "," +
+        IS_ROW_TIMESTAMP +
+        ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, 
?)";
     private static final String INSERT_COLUMN_ALTER_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + 
SYSTEM_CATALOG_TABLE + "\"( " +
             TENANT_ID + "," +
@@ -360,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 + "," +
@@ -711,10 +730,10 @@ public class MetaDataClient {
                     byte[] cf= colRef.getFamily();
                     byte[] cq= colRef.getQualifier();
                     if (cf!=null) {
-                        view.getColumnFamily(cf).getColumn(cq);
+                        
view.getColumnFamily(cf).getPColumnForColumnQualifier(cq);
                     }
                     else {
-                        view.getColumn( Bytes.toString(cq));
+                        view.getPColumnForColumnQualifier(cq);
                     }
                 } catch (ColumnNotFoundException e) { // Ignore this index and 
continue with others
                     containsAllReqdCols = false;
@@ -730,7 +749,7 @@ 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 e) { // Ignore this index 
and continue with others
                         containsAllReqdCols = false;
                         break;
@@ -788,8 +807,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();
     }
@@ -808,7 +832,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, EncodedCQCounter encodedColumnQualifier) throws SQLException {
         try {
             ColumnName columnDefName = def.getColumnDefName();
             SortOrder sortOrder = def.getSortOrder();
@@ -856,15 +880,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.getValue());
             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());
@@ -1817,7 +1840,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();
@@ -1946,7 +1969,7 @@ public class MetaDataClient {
                 columns = newArrayListWithExpectedSize(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()) {
@@ -1991,6 +2014,75 @@ 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 = connection.getTable(new 
PTableKey(null, physicalNames.get(0).getString()));
+                    storageScheme = viewPhysicalTable.getStorageScheme();
+                    if (storageScheme == StorageScheme.ENCODED_COLUMN_NAMES) {
+                        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 (HBaseAdmin admin = 
connection.getQueryServices().getAdmin()) {
+                    try {
+                        admin.getTableDescriptor(tableNameBytes);
+                    } catch (org.apache.hadoop.hbase.TableNotFoundException e) 
{
+                        tableExists = false;
+                    }
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                } catch (UnsupportedOperationException e) {
+                    //FIXME: samarth I am not sure about this.
+                }
+                if (parent != null) {
+                    storageScheme = parent.getStorageScheme();
+                } else if (tableExists) {
+                    storageScheme = StorageScheme.NON_ENCODED_COLUMN_NAMES;
+                } else {
+                    storageScheme = StorageScheme.ENCODED_COLUMN_NAMES;
+                }
+                if (storageScheme == StorageScheme.ENCODED_COLUMN_NAMES) {
+                    cqCounter = new 
EncodedCQCounter(ENCODED_CQ_COUNTER_INITIAL_VALUE);
+                }
+            }
+            
+            Integer initialCounterValue = cqCounter.getValue();
             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...
@@ -2009,7 +2101,13 @@ public class MetaDataClient {
                                 
.setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                     }
                 }
-                PColumn column = newColumn(position++, colDef, pkConstraint, 
defaultFamilyName, false);
+                ColumnName columnDefName = colDef.getColumnDefName();
+                PColumn column = null;
+                column = newColumn(position++, colDef, pkConstraint, 
defaultFamilyName, false, cqCounter);
+                if (incrementEncodedCQCounter(storageScheme, pkConstraint, 
colDef, columnDefName)) {
+                    cqCounter.increment();
+                }
+                String cf = column.getFamilyName() != null ? 
column.getFamilyName().getString() : null;
                 if (SchemaUtil.isPKColumn(column)) {
                     // TODO: remove this constraint?
                     if (pkColumnsIterator.hasNext() && 
!column.getName().getString().equals(pkColumnsIterator.next().getFirst().getColumnName()))
 {
@@ -2046,6 +2144,40 @@ public class MetaDataClient {
                         column.getFamilyName());
                 }
             }
+            
+            if (storageScheme == StorageScheme.ENCODED_COLUMN_NAMES) {
+                // 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;
+                // For local indexes and indexes on views, pass on the the 
tenant id since all their meta-data rows have
+                // tenant ids in there.
+                boolean sharedIndex = tableType == PTableType.INDEX && 
(indexType == IndexType.LOCAL || parent.getType() == PTableType.VIEW);
+                String tenantIdToUse = connection.getTenantId() != null && 
sharedIndex ? connection.getTenantId().getString() : null;
+                //TODO: samarth I think we can safely use the default column 
family here. I don't think we need to update counter for a view.
+                String familyName = QueryConstants.DEFAULT_COLUMN_FAMILY;
+                // 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 || cqCounter.getValue() != 
initialCounterValue) {
+                    try (PreparedStatement linkStatement = 
connection.prepareStatement(UPDATE_ENCODED_COLUMN_COUNTER)) {
+                        linkStatement.setString(1, tenantIdToUse);
+                        linkStatement.setString(2, schemaNameToUse);
+                        linkStatement.setString(3, tableNameToUse);
+                        linkStatement.setString(4, familyName);
+                        linkStatement.setInt(5, cqCounter.getValue());
+                        linkStatement.execute();
+                    }
+                    if (tableType == VIEW) {
+                        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();
+                    }
+                }
+            }
+            
             // 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)
@@ -2127,13 +2259,14 @@ 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, null, 
null,
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == 
null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, 
indexId, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, 
isAppendOnlySchema);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, 
indexId, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, 
isAppendOnlySchema, StorageScheme.NON_ENCODED_COLUMN_NAMES, 
PTable.EncodedCQCounter.NULL_COUNTER);
                 connection.addTable(table, 
MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             } else if (tableType == PTableType.INDEX && indexId == null) {
                 if (tableProps.get(HTableDescriptor.MAX_FILESIZE) == null) {
@@ -2274,6 +2407,7 @@ public class MetaDataClient {
                 tableUpsert.setString(24, autoPartitionSeq);
             }
             tableUpsert.setBoolean(25, isAppendOnlySchema);
+            tableUpsert.setByte(26, storageScheme.getSerializedValue()); 
//TODO: samarth should there be a null check here?
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2369,12 +2503,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,
                         dataTableName == null ? null : newSchemaName, 
dataTableName == null ? null : PNameFactory.newName(dataTableName), 
Collections.<PTable>emptyList(), isImmutableRows,
                         physicalNames, defaultFamilyName == null ? null : 
PNameFactory.newName(defaultFamilyName), viewStatement, 
Boolean.TRUE.equals(disableWAL), multiTenant, storeNulls, viewType,
-                        indexId, indexType, rowKeyOrderOptimizable, 
transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, 
isAppendOnlySchema);
+                        indexId, indexType, rowKeyOrderOptimizable, 
transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, 
isAppendOnlySchema, storageScheme, cqCounterToBe);
                 result = new MetaDataMutationResult(code, 
result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -2384,6 +2524,15 @@ public class MetaDataClient {
         }
     }
 
+    private static boolean incrementEncodedCQCounter(StorageScheme 
storageScheme, PrimaryKeyConstraint pkConstraint,
+            ColumnDef colDef, ColumnName columnDefName) {
+        return storageScheme == StorageScheme.ENCODED_COLUMN_NAMES && 
!(colDef.isPK() || (pkConstraint != null && 
pkConstraint.getColumnWithSortOrder(columnDefName) != null));
+    }
+    
+    private static boolean incrementEncodedCQCounter(StorageScheme 
storageScheme, ColumnDef colDef) {
+        return storageScheme == StorageScheme.ENCODED_COLUMN_NAMES && 
!colDef.isPK();
+    }
+
     private byte[][] getSplitKeys(List<HRegionLocation> allTableRegions) {
         if(allTableRegions.size() == 1) return null;
         byte[][] splitKeys = new byte[allTableRegions.size()-1][];
@@ -2820,7 +2969,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);
                             }
@@ -2831,7 +2980,7 @@ public class MetaDataClient {
                     }
                     else {
                         try {
-                            table.getColumn(columnName);
+                            table.getPColumnForColumnName(columnName);
                             if (!ifNotExists) {
                                 throw new 
ColumnAlreadyExistsException(schemaName, tableName, columnName);
                             }
@@ -2975,7 +3124,17 @@ public class MetaDataClient {
                 List<PColumn> columns = 
Lists.newArrayListWithExpectedSize(columnDefs.size());
                 Set<String> colFamiliesForPColumnsToBeAdded = new 
LinkedHashSet<>();
                 Set<String> families = new LinkedHashSet<>();
+                PTable tableForCQCounters = null;
+                EncodedCQCounter cqCounterToUse = NULL_COUNTER;
+                StorageScheme storageScheme = table.getStorageScheme();
+                Integer initialCounterValue = null;
                 if (columnDefs.size() > 0 ) {
+                    //FIXME: samarth change this to fetch table from server if 
client cache doesn't have it. What about local indexes?
+                    //FIXME: samarth fix this mess of getting table names from 
connection
+                    //TODO: samarth should these be guarded by storage scheme 
check. Better to have the map always available. immutable empty for views and 
non encoded.
+                    tableForCQCounters = tableType == PTableType.VIEW ? 
connection.getTable(new PTableKey(null, table.getPhysicalName().getString())) : 
table;
+                    cqCounterToUse = tableForCQCounters.getEncodedCQCounter();
+                    initialCounterValue = cqCounterToUse.getValue();
                     try (PreparedStatement colUpsert = 
connection.prepareStatement(INSERT_COLUMN_ALTER_TABLE)) {
                         short nextKeySeq = SchemaUtil.getMaxKeySeq(table);
                         for( ColumnDef colDef : columnDefs) {
@@ -2995,11 +3154,14 @@ public class MetaDataClient {
                                 throw new 
SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_CREATE_ONLY)
                                 
.setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                             }
-                            PColumn column = newColumn(position++, colDef, 
PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : 
table.getDefaultFamilyName().getString(), true);
+                            PColumn column = newColumn(position++, colDef, 
PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : 
table.getDefaultFamilyName().getString(), true, cqCounterToUse);
                             columns.add(column);
+                            if (incrementEncodedCQCounter(storageScheme, 
colDef)) {
+                                cqCounterToUse.increment();
+                            }
                             String pkName = null;
                             Short keySeq = null;
-
+                            
                             // TODO: support setting properties on other 
families?
                             if (column.getFamilyName() == null) {
                                 ++numPkColumnsAdded;
@@ -3011,9 +3173,9 @@ 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());
@@ -3033,7 +3195,7 @@ public class MetaDataClient {
                                         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_COUNTER);
                                         addColumnMutation(schemaName, 
index.getTableName().getString(), indexColumn, colUpsert, 
index.getParentTableName().getString(), index.getPKName() == null ? null : 
index.getPKName().getString(), ++nextIndexKeySeq, index.getBucketNum() != null);
                                     }
                                 }
@@ -3069,6 +3231,7 @@ 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);
@@ -3080,6 +3243,37 @@ 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;
+                //TODO: samarth I am not sure this is going to work on server 
side. But for now lets add these mutations here.
+                if (cqCounterToUse.getValue() != initialCounterValue) {
+                    PreparedStatement linkStatement;
+                    //TODO: samarth i don't think we need the shared index 
check here.
+                    //if (!sharedIndex) {
+                        linkStatement = 
connection.prepareStatement(UPDATE_ENCODED_COLUMN_COUNTER);
+                        //TODO: samarth should be ok to use the default column 
family here.    
+                        String familyName = 
QueryConstants.DEFAULT_COLUMN_FAMILY;
+                        linkStatement.setString(1, tenantIdToUse);
+                        linkStatement.setString(2, 
tableForCQCounters.getSchemaName().getString());
+                        linkStatement.setString(3, 
tableForCQCounters.getTableName().getString());
+                        linkStatement.setString(4, familyName);
+                        linkStatement.setInt(5, cqCounterToUse.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;
 
@@ -3310,8 +3504,7 @@ public class MetaDataClient {
                     Set<ColumnReference> coveredColumns = 
indexMaintainer.getCoveredColumns();
                     List<PColumn> indexColumnsToDrop = 
Lists.newArrayListWithExpectedSize(columnRefs.size());
                     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
+                        ColumnReference columnToDropRef = new 
ColumnReference(columnToDrop.getFamilyName().getBytes(), 
EncodedColumnsUtil.getColumnQualifier(columnToDrop, index));
                         if (indexColumns.contains(columnToDropRef)) {
                             if (index.getViewIndexId()==null) 
                                 indexesToDrop.add(new TableRef(index));
@@ -3319,7 +3512,7 @@ public class MetaDataClient {
                         } 
                         else if (coveredColumns.contains(columnToDropRef)) {
                             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()));
@@ -3403,13 +3596,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/450dbc59/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..a2fadac 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,11 @@ public interface PColumn extends PDatum {
     boolean isRowTimestamp();
     
     boolean isDynamic();
+    
+    /**
+     * @return name of the HBase column qualifier
+     * TODO: samarth I think we should should change this to return byte[] 
array.
+     * Then we won't have to worry about calling SchemaUtil... everywhere 
+     */
+    Integer getEncodedColumnQualifier();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450dbc59/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/450dbc59/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/450dbc59/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 a556f76..d7d2e0c 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();
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450dbc59/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
----------------------------------------------------------------------
diff --git 
a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
index 67a2714..46a5305 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PMetaDataImpl.java
@@ -423,7 +423,7 @@ public class PMetaDataImpl implements PMetaData {
             if (familyName == null) {
                 column = 
table.getPKColumn(columnToRemove.getName().getString());
             } else {
-                column = 
table.getColumnFamily(familyName).getColumn(columnToRemove.getName().getString());
+                column = 
table.getColumnFamily(familyName).getPColumnForColumnName(columnToRemove.getName().getString());
             }
             int positionOffset = 0;
             int position = column.getPosition();
@@ -438,7 +438,7 @@ public class PMetaDataImpl implements PMetaData {
             // Update position of columns that follow removed column
             for (int i = position+1; i < oldColumns.size(); i++) {
                 PColumn oldColumn = oldColumns.get(i);
-                PColumn newColumn = new PColumnImpl(oldColumn.getName(), 
oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), 
oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, 
oldColumn.getSortOrder(), oldColumn.getArraySize(), 
oldColumn.getViewConstant(), oldColumn.isViewReferenced(), null, 
oldColumn.isRowTimestamp(), oldColumn.isDynamic());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), 
oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), 
oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, 
oldColumn.getSortOrder(), oldColumn.getArraySize(), 
oldColumn.getViewConstant(), oldColumn.isViewReferenced(), null, 
oldColumn.isRowTimestamp(), oldColumn.isDynamic(), 
oldColumn.getEncodedColumnQualifier());
                 columns.add(newColumn);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450dbc59/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
index 0e1337c..8df6a95 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PName.java
@@ -83,6 +83,32 @@ public interface PName {
             return 0;
         }
     };
+    public static PName ENCODED_EMPTY_COLUMN_NAME = new PName() {
+        @Override
+        public String getString() {
+            return String.valueOf(QueryConstants.ENCODED_EMPTY_COLUMN_NAME);
+        }
+
+        @Override
+        public byte[] getBytes() {
+            return QueryConstants.ENCODED_EMPTY_COLUMN_BYTES;
+        }
+        
+        @Override
+        public String toString() {
+            return getString();
+        }
+
+        @Override
+        public ImmutableBytesPtr getBytesPtr() {
+            return QueryConstants.ENCODED_EMPTY_COLUMN_BYTES_PTR;
+        }
+
+        @Override
+        public int getEstimatedSize() {
+            return 0;
+        }
+    };
     /**
      * Get the client-side, normalized name as referenced
      * in a SQL statement.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/450dbc59/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java 
b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index 344dc2c..e230cac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
@@ -19,13 +19,13 @@ package org.apache.phoenix.schema;
 
 import java.util.List;
 
+import javax.annotation.Nullable;
+
 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.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.query.ConnectionQueryServices;
-import org.apache.phoenix.schema.stats.PTableStats;
 
 
 /**
@@ -131,7 +131,7 @@ public interface PTable extends PMetaDataEntity {
          * Link from a view to its parent table
          */
         PARENT_TABLE((byte)3);
-
+        
         private final byte[] byteValue;
         private final byte serializedValue;
 
@@ -155,6 +155,34 @@ public interface PTable extends PMetaDataEntity {
             return LinkType.values()[serializedValue-1];
         }
     }
+    
+    public enum StorageScheme {
+        ENCODED_COLUMN_NAMES((byte)1),
+        NON_ENCODED_COLUMN_NAMES((byte)2);
+
+        private final byte[] byteValue;
+        private final byte serializedValue;
+
+        StorageScheme(byte serializedValue) {
+            this.serializedValue = serializedValue;
+            this.byteValue = Bytes.toBytes(this.name());
+        }
+
+        public byte[] getBytes() {
+            return byteValue;
+        }
+
+        public byte getSerializedValue() {
+            return this.serializedValue;
+        }
+
+        public static StorageScheme fromSerializedValue(byte serializedValue) {
+            if (serializedValue < 1 || serializedValue > 
StorageScheme.values().length) {
+                return null;
+            }
+            return StorageScheme.values()[serializedValue-1];
+        }
+    }
 
     long getTimeStamp();
     long getSequenceNumber();
@@ -210,7 +238,16 @@ public interface PTable extends PMetaDataEntity {
      * can be found
      * @throws AmbiguousColumnException if multiple columns are found with the 
given name
      */
-    PColumn getColumn(String name) throws ColumnNotFoundException, 
AmbiguousColumnException;
+    PColumn getPColumnForColumnName(String name) throws 
ColumnNotFoundException, AmbiguousColumnException;
+    
+    /**
+     * Get the column with the given column qualifier.
+     * @param column qualifier bytes
+     * @return the PColumn with the given column qualifier
+     * @throws ColumnNotFoundException if no column with the given column 
qualifier can be found
+     * @throws AmbiguousColumnException if multiple columns are found with the 
given column qualifier
+     */
+    PColumn getPColumnForColumnQualifier(byte[] cq) throws 
ColumnNotFoundException, AmbiguousColumnException; 
     
     /**
      * Get the PK column with the given name.
@@ -345,7 +382,6 @@ public interface PTable extends PMetaDataEntity {
      */
     int getRowTimestampColPos();
     long getUpdateCacheFrequency();
-
     boolean isNamespaceMapped();
     
     /**
@@ -359,4 +395,49 @@ public interface PTable extends PMetaDataEntity {
      * you are also not allowed to delete the table  
      */
     boolean isAppendOnlySchema();
+    StorageScheme getStorageScheme();
+    EncodedCQCounter getEncodedCQCounter();
+    
+    /**
+     * Wrapper around {@link java.lang.Integer} to help track and update 
counter values.
+     */
+    public class EncodedCQCounter {
+        
+        @Nullable private Integer counter;
+        public static final EncodedCQCounter NULL_COUNTER = new 
EncodedCQCounter(null); 
+        
+        public EncodedCQCounter(Integer initialValue) {
+            counter = initialValue;
+        }
+        
+        @Nullable
+        public Integer getValue() {
+            return counter;
+        }
+        
+        public void increment() {
+            if (counter != null) {
+                counter++;
+            }
+        }
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + ((counter == null) ? 0 : 
counter.hashCode());
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            EncodedCQCounter other = (EncodedCQCounter)obj;
+            if (counter == null) {
+                if (other.counter != null) return false;
+            } else if (!counter.equals(other.counter)) return false;
+            return true;
+        }
+    }
 }

Reply via email to