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/23 01:01:27 UTC

[2/2] phoenix git commit: Introduce the notion of encoding schemes and storage schemes

Introduce the notion of encoding schemes and storage schemes


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

Branch: refs/heads/encodecolumns2
Commit: 3d826211daa35404e771b62fa90389b5fa4d36db
Parents: b9228a0
Author: Samarth <sa...@salesforce.com>
Authored: Tue Nov 22 17:01:15 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Tue Nov 22 17:01:15 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/compile/FromCompiler.java    |   3 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   3 +-
 .../compile/TupleProjectionCompiler.java        |   6 +-
 .../apache/phoenix/compile/UnionCompiler.java   |   3 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  15 +-
 .../coprocessor/generated/PTableProtos.java     | 336 ++++++++++++-------
 .../apache/phoenix/index/IndexMaintainer.java   |   2 +-
 .../phoenix/iterate/BaseResultIterators.java    |   2 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   2 +
 .../apache/phoenix/query/QueryConstants.java    |   4 +-
 .../apache/phoenix/schema/DelegateTable.java    |   5 +
 .../apache/phoenix/schema/MetaDataClient.java   |  22 +-
 .../org/apache/phoenix/schema/PColumnImpl.java  |   6 +-
 .../java/org/apache/phoenix/schema/PTable.java  | 176 +++++++++-
 .../org/apache/phoenix/schema/PTableImpl.java   |  56 ++--
 .../apache/phoenix/util/EncodedColumnsUtil.java |   1 -
 .../java/org/apache/phoenix/util/ScanUtil.java  |   2 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   4 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   3 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   3 +-
 .../apache/phoenix/util/PhoenixRuntimeTest.java |  12 +-
 phoenix-protocol/src/main/PTable.proto          |   5 +-
 22 files changed, 496 insertions(+), 175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
index f5df980..d4ccfff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/FromCompiler.java
@@ -71,6 +71,7 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
@@ -787,7 +788,7 @@ public class FromCompiler {
                     MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
                     Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
                     false, null, null, null, false, false, 0, 0L, SchemaUtil
-                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
+                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
index 489b993..bc2c7df 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/JoinCompiler.java
@@ -76,6 +76,7 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
@@ -1312,7 +1313,7 @@ public class JoinCompiler {
                 left.isMultiTenant(), left.getStoreNulls(), left.getViewType(), left.getViewIndexId(),
                 left.getIndexType(), left.rowKeyOrderOptimizable(), left.isTransactional(),
                 left.getUpdateCacheFrequency(), left.getIndexDisableTimestamp(), left.isNamespaceMapped(), 
-                left.getAutoPartitionSeqName(), left.isAppendOnlySchema(), StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
+                left.getAutoPartitionSeqName(), left.isAppendOnlySchema(), StorageScheme.NON_ENCODED_COLUMN_NAMES, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
index c472039..79e2971 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/TupleProjectionCompiler.java
@@ -46,6 +46,7 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.EncodedCQCounter;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
@@ -158,7 +159,7 @@ public class TupleProjectionCompiler {
                 null, null, table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
                 table.getViewIndexId(),
                 table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), 
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
+                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
     
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
@@ -185,13 +186,14 @@ public class TupleProjectionCompiler {
         if (storageScheme != StorageScheme.NON_ENCODED_COLUMN_NAMES) {
             cqCounter = EncodedCQCounter.copy(table.getEncodedCQCounter());
         }
+        
         return PTableImpl.makePTable(table.getTenantId(), PROJECTED_TABLE_SCHEMA, table.getName(), PTableType.PROJECTED,
                 null, table.getTimeStamp(), table.getSequenceNumber(), table.getPKName(),
                 retainPKColumns ? table.getBucketNum() : null, projectedColumns, null, null,
                 Collections.<PTable> emptyList(), table.isImmutableRows(), Collections.<PName> emptyList(), null, null,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
                 table.getViewIndexId(), null, table.rowKeyOrderOptimizable(), table.isTransactional(),
-                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), storageScheme, cqCounter);
+                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), storageScheme, table.getEncodingScheme(), cqCounter);
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
index d779df1..7f2fbb3 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UnionCompiler.java
@@ -39,6 +39,7 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.SchemaUtil;
@@ -99,7 +100,7 @@ public class UnionCompiler {
             null, null, projectedColumns, null, null, null, true, null, null, null, true,
             true, true, null, null, null, false, false, 0, 0L,
             SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
-                statement.getConnection().getQueryServices().getProps()), null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
+                statement.getConnection().getQueryServices().getProps()), null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);
         return tableRef;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index 93a87ea..76a1e30 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -36,6 +36,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAM
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODED_COLUMN_QUALIFIER_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES;
@@ -194,6 +195,7 @@ import org.apache.phoenix.schema.PTable;
 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.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
@@ -288,6 +290,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue AUTO_PARTITION_SEQ_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, AUTO_PARTITION_SEQ_BYTES);
     private static final KeyValue APPEND_ONLY_SCHEMA_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, APPEND_ONLY_SCHEMA_BYTES);
     private static final KeyValue STORAGE_SCHEME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, STORAGE_SCHEME_BYTES);
+    private static final KeyValue ENCODING_SCHEME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ENCODING_SCHEME_BYTES);
     
     private static final List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
             EMPTY_KEYVALUE_KV,
@@ -315,7 +318,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             IS_NAMESPACE_MAPPED_KV,
             AUTO_PARTITION_SEQ_KV,
             APPEND_ONLY_SCHEMA_KV,
-            STORAGE_SCHEME_KV
+            STORAGE_SCHEME_KV,
+            ENCODING_SCHEME_KV
             );
     static {
         Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
@@ -346,6 +350,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int AUTO_PARTITION_SEQ_INDEX = TABLE_KV_COLUMNS.indexOf(AUTO_PARTITION_SEQ_KV);
     private static final int APPEND_ONLY_SCHEMA_INDEX = TABLE_KV_COLUMNS.indexOf(APPEND_ONLY_SCHEMA_KV);
     private static final int STORAGE_SCHEME_INDEX = TABLE_KV_COLUMNS.indexOf(STORAGE_SCHEME_KV);
+    private static final int QUALIFIER_ENCODING_SCHEME_INDEX = TABLE_KV_COLUMNS.indexOf(ENCODING_SCHEME_KV);
 
     // KeyValues for Column
     private static final KeyValue DECIMAL_DIGITS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
@@ -947,7 +952,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         StorageScheme storageScheme = storageSchemeKv == null ? StorageScheme.NON_ENCODED_COLUMN_NAMES : StorageScheme
                 .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
                         storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength()));
-
+        Cell encodingSchemeKv = tableKeyValues[QUALIFIER_ENCODING_SCHEME_INDEX];
+        QualifierEncodingScheme encodingScheme = encodingSchemeKv == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : QualifierEncodingScheme
+                .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
+                    encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
+        
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();
         List<PName> physicalTables = Lists.newArrayList();
@@ -992,7 +1001,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 pkName, saltBucketNum, columns, parentSchemaName, parentTableName, indexes, isImmutableRows, physicalTables, defaultFamilyName,
                 viewStatement, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType,
                 rowKeyOrderOptimizable, transactional, updateCacheFrequency, baseColumnCount,
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, cqCounter);
+                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, encodingScheme, cqCounter);
     }
     
     private boolean isQualifierCounterKV(Cell kv) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 2f0c00b..ead928f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -270,15 +270,15 @@ public final class PTableProtos {
      */
     boolean getIsDynamic();
 
-    // optional int32 columnQualifier = 15;
+    // optional int32 encodedColumnQualifier = 15;
     /**
-     * <code>optional int32 columnQualifier = 15;</code>
+     * <code>optional int32 encodedColumnQualifier = 15;</code>
      */
-    boolean hasColumnQualifier();
+    boolean hasEncodedColumnQualifier();
     /**
-     * <code>optional int32 columnQualifier = 15;</code>
+     * <code>optional int32 encodedColumnQualifier = 15;</code>
      */
-    int getColumnQualifier();
+    int getEncodedColumnQualifier();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -403,7 +403,7 @@ public final class PTableProtos {
             }
             case 120: {
               bitField0_ |= 0x00004000;
-              columnQualifier_ = input.readInt32();
+              encodedColumnQualifier_ = input.readInt32();
               break;
             }
           }
@@ -724,20 +724,20 @@ public final class PTableProtos {
       return isDynamic_;
     }
 
-    // optional int32 columnQualifier = 15;
-    public static final int COLUMNQUALIFIER_FIELD_NUMBER = 15;
-    private int columnQualifier_;
+    // optional int32 encodedColumnQualifier = 15;
+    public static final int ENCODEDCOLUMNQUALIFIER_FIELD_NUMBER = 15;
+    private int encodedColumnQualifier_;
     /**
-     * <code>optional int32 columnQualifier = 15;</code>
+     * <code>optional int32 encodedColumnQualifier = 15;</code>
      */
-    public boolean hasColumnQualifier() {
+    public boolean hasEncodedColumnQualifier() {
       return ((bitField0_ & 0x00004000) == 0x00004000);
     }
     /**
-     * <code>optional int32 columnQualifier = 15;</code>
+     * <code>optional int32 encodedColumnQualifier = 15;</code>
      */
-    public int getColumnQualifier() {
-      return columnQualifier_;
+    public int getEncodedColumnQualifier() {
+      return encodedColumnQualifier_;
     }
 
     private void initFields() {
@@ -755,7 +755,7 @@ public final class PTableProtos {
       expression_ = "";
       isRowTimestamp_ = false;
       isDynamic_ = false;
-      columnQualifier_ = 0;
+      encodedColumnQualifier_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -832,7 +832,7 @@ public final class PTableProtos {
         output.writeBool(14, isDynamic_);
       }
       if (((bitField0_ & 0x00004000) == 0x00004000)) {
-        output.writeInt32(15, columnQualifier_);
+        output.writeInt32(15, encodedColumnQualifier_);
       }
       getUnknownFields().writeTo(output);
     }
@@ -901,7 +901,7 @@ public final class PTableProtos {
       }
       if (((bitField0_ & 0x00004000) == 0x00004000)) {
         size += com.google.protobuf.CodedOutputStream
-          .computeInt32Size(15, columnQualifier_);
+          .computeInt32Size(15, encodedColumnQualifier_);
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -996,10 +996,10 @@ public final class PTableProtos {
         result = result && (getIsDynamic()
             == other.getIsDynamic());
       }
-      result = result && (hasColumnQualifier() == other.hasColumnQualifier());
-      if (hasColumnQualifier()) {
-        result = result && (getColumnQualifier()
-            == other.getColumnQualifier());
+      result = result && (hasEncodedColumnQualifier() == other.hasEncodedColumnQualifier());
+      if (hasEncodedColumnQualifier()) {
+        result = result && (getEncodedColumnQualifier()
+            == other.getEncodedColumnQualifier());
       }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
@@ -1070,9 +1070,9 @@ public final class PTableProtos {
         hash = (37 * hash) + ISDYNAMIC_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getIsDynamic());
       }
-      if (hasColumnQualifier()) {
-        hash = (37 * hash) + COLUMNQUALIFIER_FIELD_NUMBER;
-        hash = (53 * hash) + getColumnQualifier();
+      if (hasEncodedColumnQualifier()) {
+        hash = (37 * hash) + ENCODEDCOLUMNQUALIFIER_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodedColumnQualifier();
       }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
@@ -1211,7 +1211,7 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00001000);
         isDynamic_ = false;
         bitField0_ = (bitField0_ & ~0x00002000);
-        columnQualifier_ = 0;
+        encodedColumnQualifier_ = 0;
         bitField0_ = (bitField0_ & ~0x00004000);
         return this;
       }
@@ -1300,7 +1300,7 @@ public final class PTableProtos {
         if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
           to_bitField0_ |= 0x00004000;
         }
-        result.columnQualifier_ = columnQualifier_;
+        result.encodedColumnQualifier_ = encodedColumnQualifier_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1363,8 +1363,8 @@ public final class PTableProtos {
         if (other.hasIsDynamic()) {
           setIsDynamic(other.getIsDynamic());
         }
-        if (other.hasColumnQualifier()) {
-          setColumnQualifier(other.getColumnQualifier());
+        if (other.hasEncodedColumnQualifier()) {
+          setEncodedColumnQualifier(other.getEncodedColumnQualifier());
         }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
@@ -1966,35 +1966,35 @@ public final class PTableProtos {
         return this;
       }
 
-      // optional int32 columnQualifier = 15;
-      private int columnQualifier_ ;
+      // optional int32 encodedColumnQualifier = 15;
+      private int encodedColumnQualifier_ ;
       /**
-       * <code>optional int32 columnQualifier = 15;</code>
+       * <code>optional int32 encodedColumnQualifier = 15;</code>
        */
-      public boolean hasColumnQualifier() {
+      public boolean hasEncodedColumnQualifier() {
         return ((bitField0_ & 0x00004000) == 0x00004000);
       }
       /**
-       * <code>optional int32 columnQualifier = 15;</code>
+       * <code>optional int32 encodedColumnQualifier = 15;</code>
        */
-      public int getColumnQualifier() {
-        return columnQualifier_;
+      public int getEncodedColumnQualifier() {
+        return encodedColumnQualifier_;
       }
       /**
-       * <code>optional int32 columnQualifier = 15;</code>
+       * <code>optional int32 encodedColumnQualifier = 15;</code>
        */
-      public Builder setColumnQualifier(int value) {
+      public Builder setEncodedColumnQualifier(int value) {
         bitField0_ |= 0x00004000;
-        columnQualifier_ = value;
+        encodedColumnQualifier_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional int32 columnQualifier = 15;</code>
+       * <code>optional int32 encodedColumnQualifier = 15;</code>
        */
-      public Builder clearColumnQualifier() {
+      public Builder clearEncodedColumnQualifier() {
         bitField0_ = (bitField0_ & ~0x00004000);
-        columnQualifier_ = 0;
+        encodedColumnQualifier_ = 0;
         onChanged();
         return this;
       }
@@ -3473,27 +3473,37 @@ public final class PTableProtos {
      */
     com.google.protobuf.ByteString getStorageScheme();
 
-    // repeated .EncodedCQCounter encodedCQCounters = 35;
+    // optional bytes encodingScheme = 35;
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    boolean hasEncodingScheme();
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    com.google.protobuf.ByteString getEncodingScheme();
+
+    // repeated .EncodedCQCounter encodedCQCounters = 36;
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> 
         getEncodedCQCountersList();
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index);
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     int getEncodedCQCountersCount();
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
         getEncodedCQCountersOrBuilderList();
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
         int index);
@@ -3731,9 +3741,14 @@ public final class PTableProtos {
               break;
             }
             case 282: {
-              if (!((mutable_bitField1_ & 0x00000002) == 0x00000002)) {
+              bitField0_ |= 0x40000000;
+              encodingScheme_ = input.readBytes();
+              break;
+            }
+            case 290: {
+              if (!((mutable_bitField1_ & 0x00000004) == 0x00000004)) {
                 encodedCQCounters_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter>();
-                mutable_bitField1_ |= 0x00000002;
+                mutable_bitField1_ |= 0x00000004;
               }
               encodedCQCounters_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.PARSER, extensionRegistry));
               break;
@@ -3755,7 +3770,7 @@ public final class PTableProtos {
         if (((mutable_bitField0_ & 0x00020000) == 0x00020000)) {
           physicalNames_ = java.util.Collections.unmodifiableList(physicalNames_);
         }
-        if (((mutable_bitField1_ & 0x00000002) == 0x00000002)) {
+        if (((mutable_bitField1_ & 0x00000004) == 0x00000004)) {
           encodedCQCounters_ = java.util.Collections.unmodifiableList(encodedCQCounters_);
         }
         this.unknownFields = unknownFields.build();
@@ -4433,36 +4448,52 @@ public final class PTableProtos {
       return storageScheme_;
     }
 
-    // repeated .EncodedCQCounter encodedCQCounters = 35;
-    public static final int ENCODEDCQCOUNTERS_FIELD_NUMBER = 35;
+    // optional bytes encodingScheme = 35;
+    public static final int ENCODINGSCHEME_FIELD_NUMBER = 35;
+    private com.google.protobuf.ByteString encodingScheme_;
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    public boolean hasEncodingScheme() {
+      return ((bitField0_ & 0x40000000) == 0x40000000);
+    }
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    public com.google.protobuf.ByteString getEncodingScheme() {
+      return encodingScheme_;
+    }
+
+    // repeated .EncodedCQCounter encodedCQCounters = 36;
+    public static final int ENCODEDCQCOUNTERS_FIELD_NUMBER = 36;
     private java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> encodedCQCounters_;
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> getEncodedCQCountersList() {
       return encodedCQCounters_;
     }
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
         getEncodedCQCountersOrBuilderList() {
       return encodedCQCounters_;
     }
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     public int getEncodedCQCountersCount() {
       return encodedCQCounters_.size();
     }
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index) {
       return encodedCQCounters_.get(index);
     }
     /**
-     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
      */
     public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
         int index) {
@@ -4503,6 +4534,7 @@ public final class PTableProtos {
       isAppendOnlySchema_ = false;
       parentNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+      encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
       encodedCQCounters_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
@@ -4670,8 +4702,11 @@ public final class PTableProtos {
       if (((bitField0_ & 0x20000000) == 0x20000000)) {
         output.writeBytes(34, storageScheme_);
       }
+      if (((bitField0_ & 0x40000000) == 0x40000000)) {
+        output.writeBytes(35, encodingScheme_);
+      }
       for (int i = 0; i < encodedCQCounters_.size(); i++) {
-        output.writeMessage(35, encodedCQCounters_.get(i));
+        output.writeMessage(36, encodedCQCounters_.get(i));
       }
       getUnknownFields().writeTo(output);
     }
@@ -4819,9 +4854,13 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(34, storageScheme_);
       }
+      if (((bitField0_ & 0x40000000) == 0x40000000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(35, encodingScheme_);
+      }
       for (int i = 0; i < encodedCQCounters_.size(); i++) {
         size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(35, encodedCQCounters_.get(i));
+          .computeMessageSize(36, encodedCQCounters_.get(i));
       }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
@@ -5002,6 +5041,11 @@ public final class PTableProtos {
         result = result && getStorageScheme()
             .equals(other.getStorageScheme());
       }
+      result = result && (hasEncodingScheme() == other.hasEncodingScheme());
+      if (hasEncodingScheme()) {
+        result = result && getEncodingScheme()
+            .equals(other.getEncodingScheme());
+      }
       result = result && getEncodedCQCountersList()
           .equals(other.getEncodedCQCountersList());
       result = result &&
@@ -5149,6 +5193,10 @@ public final class PTableProtos {
         hash = (37 * hash) + STORAGESCHEME_FIELD_NUMBER;
         hash = (53 * hash) + getStorageScheme().hashCode();
       }
+      if (hasEncodingScheme()) {
+        hash = (37 * hash) + ENCODINGSCHEME_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodingScheme().hashCode();
+      }
       if (getEncodedCQCountersCount() > 0) {
         hash = (37 * hash) + ENCODEDCQCOUNTERS_FIELD_NUMBER;
         hash = (53 * hash) + getEncodedCQCountersList().hashCode();
@@ -5339,9 +5387,11 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x80000000);
         storageScheme_ = com.google.protobuf.ByteString.EMPTY;
         bitField1_ = (bitField1_ & ~0x00000001);
+        encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
+        bitField1_ = (bitField1_ & ~0x00000002);
         if (encodedCQCountersBuilder_ == null) {
           encodedCQCounters_ = java.util.Collections.emptyList();
-          bitField1_ = (bitField1_ & ~0x00000002);
+          bitField1_ = (bitField1_ & ~0x00000004);
         } else {
           encodedCQCountersBuilder_.clear();
         }
@@ -5517,10 +5567,14 @@ public final class PTableProtos {
           to_bitField0_ |= 0x20000000;
         }
         result.storageScheme_ = storageScheme_;
+        if (((from_bitField1_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x40000000;
+        }
+        result.encodingScheme_ = encodingScheme_;
         if (encodedCQCountersBuilder_ == null) {
-          if (((bitField1_ & 0x00000002) == 0x00000002)) {
+          if (((bitField1_ & 0x00000004) == 0x00000004)) {
             encodedCQCounters_ = java.util.Collections.unmodifiableList(encodedCQCounters_);
-            bitField1_ = (bitField1_ & ~0x00000002);
+            bitField1_ = (bitField1_ & ~0x00000004);
           }
           result.encodedCQCounters_ = encodedCQCounters_;
         } else {
@@ -5698,11 +5752,14 @@ public final class PTableProtos {
         if (other.hasStorageScheme()) {
           setStorageScheme(other.getStorageScheme());
         }
+        if (other.hasEncodingScheme()) {
+          setEncodingScheme(other.getEncodingScheme());
+        }
         if (encodedCQCountersBuilder_ == null) {
           if (!other.encodedCQCounters_.isEmpty()) {
             if (encodedCQCounters_.isEmpty()) {
               encodedCQCounters_ = other.encodedCQCounters_;
-              bitField1_ = (bitField1_ & ~0x00000002);
+              bitField1_ = (bitField1_ & ~0x00000004);
             } else {
               ensureEncodedCQCountersIsMutable();
               encodedCQCounters_.addAll(other.encodedCQCounters_);
@@ -5715,7 +5772,7 @@ public final class PTableProtos {
               encodedCQCountersBuilder_.dispose();
               encodedCQCountersBuilder_ = null;
               encodedCQCounters_ = other.encodedCQCounters_;
-              bitField1_ = (bitField1_ & ~0x00000002);
+              bitField1_ = (bitField1_ & ~0x00000004);
               encodedCQCountersBuilder_ = 
                 com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
                    getEncodedCQCountersFieldBuilder() : null;
@@ -7494,13 +7551,49 @@ public final class PTableProtos {
         return this;
       }
 
-      // repeated .EncodedCQCounter encodedCQCounters = 35;
+      // optional bytes encodingScheme = 35;
+      private com.google.protobuf.ByteString encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public boolean hasEncodingScheme() {
+        return ((bitField1_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public com.google.protobuf.ByteString getEncodingScheme() {
+        return encodingScheme_;
+      }
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public Builder setEncodingScheme(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField1_ |= 0x00000002;
+        encodingScheme_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public Builder clearEncodingScheme() {
+        bitField1_ = (bitField1_ & ~0x00000002);
+        encodingScheme_ = getDefaultInstance().getEncodingScheme();
+        onChanged();
+        return this;
+      }
+
+      // repeated .EncodedCQCounter encodedCQCounters = 36;
       private java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> encodedCQCounters_ =
         java.util.Collections.emptyList();
       private void ensureEncodedCQCountersIsMutable() {
-        if (!((bitField1_ & 0x00000002) == 0x00000002)) {
+        if (!((bitField1_ & 0x00000004) == 0x00000004)) {
           encodedCQCounters_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter>(encodedCQCounters_);
-          bitField1_ |= 0x00000002;
+          bitField1_ |= 0x00000004;
          }
       }
 
@@ -7508,7 +7601,7 @@ public final class PTableProtos {
           org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> encodedCQCountersBuilder_;
 
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> getEncodedCQCountersList() {
         if (encodedCQCountersBuilder_ == null) {
@@ -7518,7 +7611,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public int getEncodedCQCountersCount() {
         if (encodedCQCountersBuilder_ == null) {
@@ -7528,7 +7621,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index) {
         if (encodedCQCountersBuilder_ == null) {
@@ -7538,7 +7631,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder setEncodedCQCounters(
           int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter value) {
@@ -7555,7 +7648,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder setEncodedCQCounters(
           int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder builderForValue) {
@@ -7569,7 +7662,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder addEncodedCQCounters(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter value) {
         if (encodedCQCountersBuilder_ == null) {
@@ -7585,7 +7678,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder addEncodedCQCounters(
           int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter value) {
@@ -7602,7 +7695,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder addEncodedCQCounters(
           org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder builderForValue) {
@@ -7616,7 +7709,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder addEncodedCQCounters(
           int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder builderForValue) {
@@ -7630,7 +7723,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder addAllEncodedCQCounters(
           java.lang.Iterable<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> values) {
@@ -7644,12 +7737,12 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder clearEncodedCQCounters() {
         if (encodedCQCountersBuilder_ == null) {
           encodedCQCounters_ = java.util.Collections.emptyList();
-          bitField1_ = (bitField1_ & ~0x00000002);
+          bitField1_ = (bitField1_ & ~0x00000004);
           onChanged();
         } else {
           encodedCQCountersBuilder_.clear();
@@ -7657,7 +7750,7 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public Builder removeEncodedCQCounters(int index) {
         if (encodedCQCountersBuilder_ == null) {
@@ -7670,14 +7763,14 @@ public final class PTableProtos {
         return this;
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder getEncodedCQCountersBuilder(
           int index) {
         return getEncodedCQCountersFieldBuilder().getBuilder(index);
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
           int index) {
@@ -7687,7 +7780,7 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
            getEncodedCQCountersOrBuilderList() {
@@ -7698,14 +7791,14 @@ public final class PTableProtos {
         }
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder addEncodedCQCountersBuilder() {
         return getEncodedCQCountersFieldBuilder().addBuilder(
             org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.getDefaultInstance());
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder addEncodedCQCountersBuilder(
           int index) {
@@ -7713,7 +7806,7 @@ public final class PTableProtos {
             index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.getDefaultInstance());
       }
       /**
-       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
        */
       public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder> 
            getEncodedCQCountersBuilderList() {
@@ -7726,7 +7819,7 @@ public final class PTableProtos {
           encodedCQCountersBuilder_ = new com.google.protobuf.RepeatedFieldBuilder<
               org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder>(
                   encodedCQCounters_,
-                  ((bitField1_ & 0x00000002) == 0x00000002),
+                  ((bitField1_ & 0x00000004) == 0x00000004),
                   getParentForChildren(),
                   isClean());
           encodedCQCounters_ = null;
@@ -8389,7 +8482,7 @@ public final class PTableProtos {
       descriptor;
   static {
     java.lang.String[] descriptorData = {
-      "\n\014PTable.proto\032\021PGuidePosts.proto\"\277\002\n\007PC" +
+      "\n\014PTable.proto\032\021PGuidePosts.proto\"\306\002\n\007PC" +
       "olumn\022\027\n\017columnNameBytes\030\001 \002(\014\022\027\n\017family" +
       "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \002(\t\022\021\n\tmax" +
       "Length\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010nullable\030" +
@@ -8397,37 +8490,38 @@ public final class PTableProtos {
       "(\005\022\021\n\tarraySize\030\t \001(\005\022\024\n\014viewConstant\030\n " +
       "\001(\014\022\026\n\016viewReferenced\030\013 \001(\010\022\022\n\nexpressio" +
       "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\022\021\n\tisDyn" +
-      "amic\030\016 \001(\010\022\027\n\017columnQualifier\030\017 \001(\005\"\232\001\n\013" +
-      "PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006values\030\002 \003(\014",
-      "\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n\rkeyByte" +
-      "sCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001(\005\022!\n" +
-      "\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"\324\006\n\006PTa" +
-      "ble\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tableNam" +
-      "eBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.PTable" +
-      "Type\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenceNumb" +
-      "er\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pkNameByt" +
-      "es\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007columns\030\t" +
-      " \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007.PTable" +
-      "\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022dataTableNa",
-      "meBytes\030\r \001(\014\022\031\n\021defaultFamilyName\030\016 \001(\014" +
-      "\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTenant\030\020 \002(" +
-      "\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewStatement\030\022 \001" +
-      "(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010tenantId\030\024 " +
-      "\001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tindexType\030\026 " +
-      "\001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\nstoreNull" +
-      "s\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005\022\036\n\026rowK" +
-      "eyOrderOptimizable\030\032 \001(\010\022\025\n\rtransactiona" +
-      "l\030\033 \001(\010\022\034\n\024updateCacheFrequency\030\034 \001(\003\022\035\n" +
-      "\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021isNamesp",
-      "aceMapped\030\036 \001(\010\022\034\n\024autoParititonSeqName\030" +
-      "\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(\010\022\027\n\017par" +
-      "entNameBytes\030! \001(\014\022\025\n\rstorageScheme\030\" \001(" +
-      "\014\022,\n\021encodedCQCounters\030# \003(\0132\021.EncodedCQ" +
-      "Counter\"6\n\020EncodedCQCounter\022\021\n\tcolFamily" +
-      "\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A\n\nPTableType\022\n\n" +
-      "\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003" +
-      "\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.coproce" +
-      "ssor.generatedB\014PTableProtosH\001\210\001\001\240\001\001"
+      "amic\030\016 \001(\010\022\036\n\026encodedColumnQualifier\030\017 \001" +
+      "(\005\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022\016\n\006value",
+      "s\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n\r" +
+      "keyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005" +
+      " \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"" +
+      "\354\006\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016t" +
+      "ableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013" +
+      ".PTableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016seque" +
+      "nceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pk" +
+      "NameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007co" +
+      "lumns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007" +
+      ".PTable\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022data",
+      "TableNameBytes\030\r \001(\014\022\031\n\021defaultFamilyNam" +
+      "e\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013multiTena" +
+      "nt\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rviewStatem" +
+      "ent\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010tena" +
+      "ntId\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tindex" +
+      "Type\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\nst" +
+      "oreNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005\022" +
+      "\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtrans" +
+      "actional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030\034" +
+      " \001(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021i",
+      "sNamespaceMapped\030\036 \001(\010\022\034\n\024autoParititonS" +
+      "eqName\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(\010" +
+      "\022\027\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageSche" +
+      "me\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021enco" +
+      "dedCQCounters\030$ \003(\0132\021.EncodedCQCounter\"6" +
+      "\n\020EncodedCQCounter\022\021\n\tcolFamily\030\001 \002(\t\022\017\n" +
+      "\007counter\030\002 \002(\005*A\n\nPTableType\022\n\n\006SYSTEM\020\000" +
+      "\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020" +
+      "\004B@\n(org.apache.phoenix.coprocessor.gene" +
+      "ratedB\014PTableProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8439,7 +8533,7 @@ public final class PTableProtos {
           internal_static_PColumn_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PColumn_descriptor,
-              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "ColumnQualifier", });
+              new java.lang.String[] { "ColumnNameBytes", "FamilyNameBytes", "DataType", "MaxLength", "Scale", "Nullable", "Position", "SortOrder", "ArraySize", "ViewConstant", "ViewReferenced", "Expression", "IsRowTimestamp", "IsDynamic", "EncodedColumnQualifier", });
           internal_static_PTableStats_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_PTableStats_fieldAccessorTable = new
@@ -8451,7 +8545,7 @@ public final class PTableProtos {
           internal_static_PTable_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PTable_descriptor,
-              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisableTimestamp", "IsNamespaceMapped", "AutoParititonSeqName", "IsAppendOnlySchema", "ParentNameBytes", "StorageScheme", "EncodedCQCounters", });
+              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisableTimestamp", "IsNamespaceMapped", "AutoParititonSeqName", "IsAppendOnlySchema", "ParentNameBytes", "StorageScheme", "EncodingScheme", "EncodedCQCounters", });
           internal_static_EncodedCQCounter_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_EncodedCQCounter_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 7c88a25..a876a0b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -1716,7 +1716,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             return StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL;
         }
         if (usesEncodedColumnNames) {
-            return StorageScheme.ENCODED_COLUMN_NAMES;
+            return StorageScheme.COLUMNS_STORED_IN_INDIVIDUAL_CELLS;
         }
         return StorageScheme.NON_ENCODED_COLUMN_NAMES;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
index cdd9cbc..fc6e3bf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/BaseResultIterators.java
@@ -272,7 +272,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             throws SQLException {
         PTable table = context.getCurrentTable().getTable();
         StorageScheme storageScheme = table.getStorageScheme();
-        checkArgument(storageScheme == StorageScheme.ENCODED_COLUMN_NAMES,
+        checkArgument(storageScheme == StorageScheme.COLUMNS_STORED_IN_INDIVIDUAL_CELLS,
             "Method should only be used for tables using encoded column names");
         Pair<Integer, Integer> minMaxQualifiers = new Pair<>();
         for (Pair<byte[], byte[]> whereCol : context.getWhereConditionColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
index bc81447..40952cd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixDatabaseMetaData.java
@@ -322,6 +322,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     
     public static final String STORAGE_SCHEME = "STORAGE_SCHEME";
     public static final byte[] STORAGE_SCHEME_BYTES = Bytes.toBytes(STORAGE_SCHEME);
+    public static final String ENCODING_SCHEME = "ENCODING_SCHEME";
+    public static final byte[] ENCODING_SCHEME_BYTES = Bytes.toBytes(ENCODING_SCHEME);
     public static final String ENCODED_COLUMN_QUALIFIER = "COLUMN_QUALIFIER";
     public static final byte[] ENCODED_COLUMN_QUALIFIER_BYTES = Bytes.toBytes(ENCODED_COLUMN_QUALIFIER);
     public static final String COLUMN_QUALIFIER_COUNTER = "QUALIFIER_COUNTER";

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/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 e44d83e..954e78a 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
@@ -42,6 +42,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAM
 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.ENCODING_SCHEME;
 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;
@@ -315,7 +316,8 @@ public interface QueryConstants {
             APPEND_ONLY_SCHEMA + " BOOLEAN," +
             GUIDE_POSTS_WIDTH + " BIGINT," +
             ENCODED_COLUMN_QUALIFIER + " UNSIGNED_INT," +
-            STORAGE_SCHEME + " TINYINT, " + 
+            STORAGE_SCHEME + " TINYINT, " +
+            ENCODING_SCHEME + " TINYINT, " +
             COLUMN_QUALIFIER_COUNTER + " INTEGER, " +
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
             + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/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 b09fc5f..7168c5e 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
@@ -305,4 +305,9 @@ public class DelegateTable implements PTable {
     public EncodedCQCounter getEncodedCQCounter() {
         return delegate.getEncodedCQCounter();
     }
+
+    @Override
+    public QualifierEncodingScheme getEncodingScheme() {
+        return delegate.getEncodingScheme();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/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 f49dee7..bc64e4b 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
@@ -43,6 +43,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAM
 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.ENCODING_SCHEME;
 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;
@@ -193,6 +194,7 @@ 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.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
@@ -267,8 +269,9 @@ public class MetaDataClient {
                     AUTO_PARTITION_SEQ +  "," +
                     APPEND_ONLY_SCHEMA + "," +
                     GUIDE_POSTS_WIDTH + "," +
-                    STORAGE_SCHEME +
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    STORAGE_SCHEME + "," +
+                    ENCODING_SCHEME + 
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -2073,11 +2076,15 @@ public class MetaDataClient {
                     // 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;
+                    storageScheme = StorageScheme.COLUMNS_STORED_IN_INDIVIDUAL_CELLS;
                 }
                 cqCounter = storageScheme != StorageScheme.NON_ENCODED_COLUMN_NAMES ? new EncodedCQCounter() : NULL_COUNTER;
             }
             
+            //FIXME: samarth change this once we start having qualifier encoding scheme options
+            QualifierEncodingScheme encodingScheme =
+                    storageScheme != StorageScheme.NON_ENCODED_COLUMN_NAMES ? QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS
+                            : QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
             Map<String, Integer> changedCqCounters = new HashMap<>(colDefs.size());
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
@@ -2240,7 +2247,7 @@ public class MetaDataClient {
                         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, StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, StorageScheme.NON_ENCODED_COLUMN_NAMES, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
             
@@ -2394,7 +2401,8 @@ public class MetaDataClient {
             } else {
                 tableUpsert.setLong(25, guidePostsWidth);
             }
-            tableUpsert.setByte(26, storageScheme.getSerializedValue()); //TODO: samarth should there be a null check here?
+            tableUpsert.setByte(26, storageScheme.getSerializedMetadataValue()); //TODO: samarth should there be a null check here?
+            tableUpsert.setByte(27, encodingScheme.getSerializedMetadataValue());
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2499,7 +2507,7 @@ public class MetaDataClient {
                         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, storageScheme, cqCounterToBe);
+                        result.getViewIndexId(), indexType, rowKeyOrderOptimizable, transactional, updateCacheFrequency, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, encodingScheme, cqCounterToBe);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -3589,7 +3597,7 @@ public class MetaDataClient {
                                         sharedTableState.getSchemaName(), sharedTableState.getTableName(), ts,
                                         table.getColumnFamilies(), sharedTableState.getColumns(),
                                         sharedTableState.getPhysicalNames(), sharedTableState.getViewIndexId(),
-                                        table.isMultiTenant(), table.isNamespaceMapped(), table.getStorageScheme(), table.getEncodedCQCounter());
+                                        table.isMultiTenant(), table.isNamespaceMapped(), table.getStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
                                 TableRef indexTableRef = new TableRef(viewIndexTable);
                                 PName indexTableTenantId = sharedTableState.getTenantId();
                                 if (indexTableTenantId==null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/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 e6f8b71..dbb96d5 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
@@ -259,8 +259,8 @@ public class PColumnImpl implements PColumn {
         	isDynamic = column.getIsDynamic();
         }
         Integer columnQualifier = null;
-        if (column.hasColumnQualifier()) {
-            columnQualifier = column.getColumnQualifier();
+        if (column.hasEncodedColumnQualifier()) {
+            columnQualifier = column.getEncodedColumnQualifier();
         }
         return new PColumnImpl(columnName, familyName, dataType, maxLength, scale, nullable, position, sortOrder,
                 arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifier);
@@ -295,7 +295,7 @@ public class PColumnImpl implements PColumn {
         }
         builder.setIsRowTimestamp(column.isRowTimestamp());
         if (column.getEncodedColumnQualifier() != null) {
-            builder.setColumnQualifier(column.getEncodedColumnQualifier());
+            builder.setEncodedColumnQualifier(column.getEncodedColumnQualifier());
         }
         return builder.build();
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/3d826211/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 d3b11b2..afe664b 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,6 +19,7 @@ package org.apache.phoenix.schema;
 
 import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
 
+import java.nio.ByteOrder;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
@@ -163,12 +164,13 @@ public interface PTable extends PMetaDataEntity {
     }
     
     public enum StorageScheme {
-        ENCODED_COLUMN_NAMES((byte)1),
-        NON_ENCODED_COLUMN_NAMES((byte)2),
+        COLUMNS_STORED_IN_INDIVIDUAL_CELLS((byte)1),
+        NON_ENCODED_COLUMN_NAMES((byte)2),//TODO: samarth don't need this as storage scheme. Instead make it an encoding scheme
         COLUMNS_STORED_IN_SINGLE_CELL((byte)3);
 
         private final byte[] byteValue;
         private final byte serializedValue;
+        private static final ByteOrder BYTE_ORDER = ByteOrder.nativeOrder();
 
         StorageScheme(byte serializedValue) {
             this.serializedValue = serializedValue;
@@ -179,7 +181,7 @@ public interface PTable extends PMetaDataEntity {
             return byteValue;
         }
 
-        public byte getSerializedValue() {
+        public byte getSerializedMetadataValue() {
             return this.serializedValue;
         }
 
@@ -190,6 +192,171 @@ public interface PTable extends PMetaDataEntity {
             return StorageScheme.values()[serializedValue-1];
         }
     }
+    
+    public static class QualifierEncodingScheme<E> implements QualifierEncoderDecoder<E> {
+        
+        public static final QualifierEncodingScheme NON_ENCODED_QUALIFIERS = new QualifierEncodingScheme<String>((byte)0, "NON_ENCODED_QUALIFIERS", null) {
+            @Override
+            public byte[] getEncodedBytes(String value) {
+                return Bytes.toBytes(value);
+            }
+
+            @Override
+            public String getDecodedValue(byte[] bytes) {
+                return Bytes.toString(bytes);
+            }
+
+            @Override
+            public String getDecodedValue(byte[] bytes, int offset, int length) {
+                return Bytes.toString(bytes, offset, length);
+            }
+            
+            @Override
+            public boolean isEncodeable(String value) {
+                return true;
+            }
+        };
+        public static final QualifierEncodingScheme ONE_BYTE_QUALIFIERS = new QualifierEncodingScheme<Long>((byte)1, "ONE_BYTE_QUALIFIERS", 255l) {
+            @Override
+            public byte[] getEncodedBytes(Long value) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes, int offset, int length) {
+                throw new UnsupportedOperationException();
+            }
+            
+            @Override
+            public boolean isEncodeable(Long value) {
+                return true;
+            }
+        };
+        public static final QualifierEncodingScheme TWO_BYTE_QUALIFIERS = new QualifierEncodingScheme<Long>((byte)2, "TWO_BYTE_QUALIFIERS", 65535l) {
+            @Override
+            public byte[] getEncodedBytes(Long value) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes, int offset, int length) {
+                throw new UnsupportedOperationException();
+            }
+            
+            @Override
+            public boolean isEncodeable(Long value) {
+                return true;
+            }
+        };
+        public static final QualifierEncodingScheme THREE_BYTE_QUALIFIERS = new QualifierEncodingScheme<Long>((byte)3, "THREE_BYTE_QUALIFIERS", 16777215l) {
+            @Override
+            public byte[] getEncodedBytes(Long value) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes, int offset, int length) {
+                throw new UnsupportedOperationException();
+            }
+            
+            @Override
+            public boolean isEncodeable(Long value) {
+                return true;
+            }
+        };
+        public static final QualifierEncodingScheme FOUR_BYTE_QUALIFIERS = new QualifierEncodingScheme<Long>((byte)4, "FOUR_BYTE_QUALIFIERS", 4294967295l) {
+            @Override
+            public byte[] getEncodedBytes(Long value) {
+                return Bytes.toBytes(value);
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes) {
+                return Bytes.toLong(bytes);
+            }
+
+            @Override
+            public Long getDecodedValue(byte[] bytes, int offset, int length) {
+                return Bytes.toLong(bytes, offset, length);
+            }
+            
+            @Override
+            public boolean isEncodeable(Long value) {
+                return true;
+            }
+        };
+        public static final QualifierEncodingScheme[] schemes = {NON_ENCODED_QUALIFIERS, ONE_BYTE_QUALIFIERS, TWO_BYTE_QUALIFIERS, THREE_BYTE_QUALIFIERS, FOUR_BYTE_QUALIFIERS}; 
+        private final byte[] metadataBytes;
+        private final byte metadataValue;
+        private final Long maxQualifier;
+
+        private QualifierEncodingScheme(byte serializedMetadataValue, String name, Long maxQualifier) {
+            this.metadataValue = serializedMetadataValue;
+            this.metadataBytes = Bytes.toBytes(name);
+            this.maxQualifier = maxQualifier;
+        }
+
+        public byte[] getMetadataBytes() {
+            return metadataBytes;
+        }
+
+        public byte getSerializedMetadataValue() {
+            return this.metadataValue;
+        }
+
+        public static QualifierEncodingScheme fromSerializedValue(byte serializedValue) {
+            if (serializedValue < 0 || serializedValue > schemes.length - 1) {
+                return null;
+            }
+            return schemes[serializedValue];
+        }
+        
+        public Long getMaxQualifier() {
+            return maxQualifier;
+        }
+
+        @Override
+        public byte[] getEncodedBytes(E value) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public E getDecodedValue(byte[] bytes) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public E getDecodedValue(byte[] bytes, int offset, int length) {
+            throw new UnsupportedOperationException();
+        }
+
+        @Override
+        public boolean isEncodeable(E value) {
+            throw new UnsupportedOperationException();
+        }
+    }
+    
+    interface QualifierEncoderDecoder<E> {
+        byte[] getEncodedBytes(E value);
+        E getDecodedValue(byte[] bytes);
+        E getDecodedValue(byte[] bytes, int offset, int length);
+        boolean isEncodeable(E value);
+    }
 
     long getTimeStamp();
     long getSequenceNumber();
@@ -405,6 +572,7 @@ public interface PTable extends PMetaDataEntity {
      */
     boolean isAppendOnlySchema();
     StorageScheme getStorageScheme();
+    QualifierEncodingScheme getEncodingScheme();
     EncodedCQCounter getEncodedCQCounter();
     
     /**
@@ -472,7 +640,7 @@ public interface PTable extends PMetaDataEntity {
         /**
          * 
          * @param columnFamily
-         * @return true if the counter was incrememnted, false otherwise.
+         * @return true if the counter was incremented, false otherwise.
          */
         public boolean increment(String columnFamily) {
             if (columnFamily == null) {