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/08 07:54:48 UTC

[01/13] phoenix git commit: PHOENIX-3462 BaseLocalIndexIT is changing server side properties of the shared test cluster [Forced Update!]

Repository: phoenix
Updated Branches:
  refs/heads/encodecolumns2 2611c5984 -> 045ffcbe2 (forced update)


PHOENIX-3462 BaseLocalIndexIT is changing server side properties of the shared test cluster


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

Branch: refs/heads/encodecolumns2
Commit: c030eedd16e1fbaaa7d4a444aed24dab3cbe2b2a
Parents: 9582d0e
Author: Samarth <sa...@salesforce.com>
Authored: Mon Nov 7 16:48:33 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Nov 7 16:48:33 2016 -0800

----------------------------------------------------------------------
 .../java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/c030eedd/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
index e818665..a30377b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/BaseLocalIndexIT.java
@@ -25,7 +25,7 @@ import java.util.Collection;
 import java.util.Map;
 import java.util.Properties;
 
-import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.end2end.BaseOwnClusterIT;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.PropertiesUtil;
@@ -40,7 +40,7 @@ import org.junit.runners.Parameterized.Parameters;
 import com.google.common.collect.Maps;
 
 @RunWith(Parameterized.class)
-public abstract class BaseLocalIndexIT extends ParallelStatsDisabledIT {
+public abstract class BaseLocalIndexIT extends BaseOwnClusterIT {
     protected boolean isNamespaceMapped;
     protected String schemaName;
 


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedResultTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedResultTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedResultTuple.java
new file mode 100644
index 0000000..109cfc3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedResultTuple.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.tuple;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+
+public class PositionBasedResultTuple extends BaseTuple {
+    private final EncodedColumnQualiferCellsList cells;
+    
+    public PositionBasedResultTuple(List<Cell> list) {
+        checkArgument(list instanceof EncodedColumnQualiferCellsList, "Invalid list type");
+        this.cells = (EncodedColumnQualiferCellsList)list;
+    }
+    
+    @Override
+    public void getKey(ImmutableBytesWritable ptr) {
+        Cell value = cells.getFirstCell();
+        ptr.set(value.getRowArray(), value.getRowOffset(), value.getRowLength());
+    }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
+
+    @Override
+    public KeyValue getValue(byte[] family, byte[] qualifier) {
+        int columnQualifier = EncodedColumnsUtil.getEncodedColumnQualifier(qualifier);
+        return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cells.getCellForColumnQualifier(columnQualifier));
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("keyvalues=");
+      if(this.cells == null || this.cells.isEmpty()) {
+        sb.append("NONE");
+        return sb.toString();
+      }
+      sb.append("{");
+      boolean moreThanOne = false;
+      for(Cell kv : this.cells) {
+        if(moreThanOne) {
+          sb.append(", \n");
+        } else {
+          moreThanOne = true;
+        }
+        sb.append(kv.toString()+"/value="+Bytes.toString(kv.getValueArray(), 
+          kv.getValueOffset(), kv.getValueLength()));
+      }
+      sb.append("}\n");
+      return sb.toString();
+    }
+
+    @Override
+    public int size() {
+        return cells.size();
+    }
+
+    @Override
+    public KeyValue getValue(int index) {
+        return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(index == 0 ? cells.getFirstCell() : cells.get(index));
+    }
+
+    @Override
+    public boolean getValue(byte[] family, byte[] qualifier,
+            ImmutableBytesWritable ptr) {
+        KeyValue kv = getValue(family, qualifier);
+        if (kv == null)
+            return false;
+        ptr.set(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
+        return true;
+    }
+    
+    public Iterator<Cell> getTupleIterator() {
+        return new TupleIterator(cells.iterator());
+    }
+    
+    private static class TupleIterator implements Iterator<Cell> {
+        
+        private final Iterator<Cell> delegate;
+        private TupleIterator(Iterator<Cell> delegate) {
+            this.delegate = delegate;
+        }
+        
+        @Override
+        public boolean hasNext() {
+            return delegate.hasNext();
+        }
+
+        @Override
+        public Cell next() {
+            return delegate.next();
+        }
+
+        @Override
+        public void remove() {
+            delegate.remove();
+        }
+        
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
index c28a2bf..3774837 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/ResultTuple.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import java.util.Collections;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Result;
@@ -25,25 +27,23 @@ import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.util.KeyValueUtil;
 
-
+/**
+ * 
+ * Wrapper around {@link Result} that implements Phoenix's {@link Tuple} interface.
+ *
+ */
 public class ResultTuple extends BaseTuple {
-    private Result result;
+    private final Result result;
+    public static final ResultTuple EMPTY_TUPLE = new ResultTuple(Result.create(Collections.<Cell>emptyList()));
     
     public ResultTuple(Result result) {
         this.result = result;
     }
     
-    public ResultTuple() {
-    }
-    
     public Result getResult() {
         return this.result;
     }
 
-    public void setResult(Result result) {
-        this.result = result;
-    }
-    
     @Override
     public void getKey(ImmutableBytesWritable ptr) {
         ptr.set(result.getRow());
@@ -104,4 +104,4 @@ public class ResultTuple extends BaseTuple {
         ptr.set(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
         return true;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
index 61b2a4f..e4a887b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/Tuple.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
@@ -87,4 +89,6 @@ public interface Tuple {
      * @return the current or next sequence value
      */
     public long getSequenceValue(int index);
+    
+    public void setKeyValues(List<Cell> values);
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/EncodedColumnsUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/EncodedColumnsUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/EncodedColumnsUtil.java
new file mode 100644
index 0000000..aeb4e46
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/EncodedColumnsUtil.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.util;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.StorageScheme;
+import org.apache.phoenix.schema.types.PInteger;
+
+public class EncodedColumnsUtil {
+
+    public static boolean usesEncodedColumnNames(PTable table) {
+        return usesEncodedColumnNames(table.getStorageScheme());
+    }
+    
+    public static boolean usesEncodedColumnNames(StorageScheme storageScheme) {
+        return storageScheme != null && storageScheme != StorageScheme.NON_ENCODED_COLUMN_NAMES;
+    }
+
+    public static byte[] getEncodedColumnQualifier(PColumn column) {
+        checkArgument(!SchemaUtil.isPKColumn(column), "No column qualifiers for PK columns");
+        checkArgument(!column.isDynamic(), "No encoded column qualifiers for dynamic columns");
+        return Bytes.toBytes(column.getEncodedColumnQualifier());
+    }
+    
+    public static int getEncodedColumnQualifier(byte[] bytes, int offset, int length) {
+        return Bytes.toInt(bytes, offset, length);
+    }
+    
+    public static byte[] getEncodedColumnQualifier(int value) {
+        return Bytes.toBytes(value);
+    }
+    
+    public static int getEncodedColumnQualifier(byte[] bytes) {
+        return Bytes.toInt(bytes);
+    }
+
+    public static byte[] getColumnQualifier(PColumn column, PTable table) {
+      return EncodedColumnsUtil.getColumnQualifier(column, usesEncodedColumnNames(table));
+    }
+    
+    public static void setColumns(PColumn column, PTable table, Scan scan) {
+    	if (table.getStorageScheme() == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+            // if a table storage scheme is COLUMNS_STORED_IN_SINGLE_CELL set then all columns of a column family are stored in a single cell 
+            // (with the qualifier name being same as the family name), just project the column family here
+            // so that we can calculate estimatedByteSize correctly in ProjectionCompiler 
+    		scan.addFamily(column.getFamilyName().getBytes());
+    		//scan.addColumn(column.getFamilyName().getBytes(), column.getFamilyName().getBytes());
+        }
+        else {
+        	scan.addColumn(column.getFamilyName().getBytes(), EncodedColumnsUtil.getColumnQualifier(column, table));
+        }
+    }
+    
+    public static byte[] getColumnQualifier(PColumn column, boolean encodedColumnName) {
+        checkArgument(!SchemaUtil.isPKColumn(column), "No column qualifiers for PK columns");
+        if (column.isDynamic()) { // Dynamic column names don't have encoded column names
+            return column.getName().getBytes();
+        }
+        return encodedColumnName ? getEncodedColumnQualifier(column) : column.getName().getBytes(); 
+    }
+
+    /**
+     * @return pair of byte arrays. The first part of the pair is the empty key value's column qualifier, and the second
+     *         part is the value to use for it.
+     */
+    public static Pair<byte[], byte[]> getEmptyKeyValueInfo(PTable table) {
+        return usesEncodedColumnNames(table) ? new Pair<>(QueryConstants.ENCODED_EMPTY_COLUMN_BYTES,
+                QueryConstants.ENCODED_EMPTY_COLUMN_VALUE_BYTES) : new Pair<>(QueryConstants.EMPTY_COLUMN_BYTES,
+                QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+    }
+
+    /**
+     * @return pair of byte arrays. The first part of the pair is the empty key value's column qualifier, and the second
+     *         part is the value to use for it.
+     */
+    public static Pair<byte[], byte[]> getEmptyKeyValueInfo(boolean usesEncodedColumnNames) {
+        return usesEncodedColumnNames ? new Pair<>(QueryConstants.ENCODED_EMPTY_COLUMN_BYTES,
+                QueryConstants.ENCODED_EMPTY_COLUMN_VALUE_BYTES) : new Pair<>(QueryConstants.EMPTY_COLUMN_BYTES,
+                QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+    }
+
+    public static boolean hasEncodedColumnName(PColumn column){
+        return !SchemaUtil.isPKColumn(column) && !column.isDynamic() && column.getEncodedColumnQualifier() != null;
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
index 9eb3fae..c088b25 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -18,6 +18,10 @@
 package org.apache.phoenix.util;
 
 import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_QUALIFIER;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
@@ -25,6 +29,7 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.Cell;
@@ -54,7 +59,9 @@ import org.apache.phoenix.compile.WhereCompiler;
 import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
+import org.apache.phoenix.execute.MutationState.RowMutationState;
 import org.apache.phoenix.execute.TupleProjector;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
@@ -77,21 +84,24 @@ import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.tephra.TxConstants;
 
 import com.google.common.collect.Lists;
 
-import org.apache.tephra.TxConstants;
-
 public class IndexUtil {
     public static final String INDEX_COLUMN_NAME_SEP = ":";
     public static final byte[] INDEX_COLUMN_NAME_SEP_BYTES = Bytes.toBytes(INDEX_COLUMN_NAME_SEP);
@@ -192,7 +202,7 @@ public class IndexUtil {
             throw new IllegalArgumentException("Could not find column family \"" +  indexColumnName.substring(0, pos) + "\" in index column name of \"" + indexColumnName + "\"", e);
         }
         try {
-            return family.getColumn(indexColumnName.substring(pos+1));
+            return family.getPColumnForColumnName(indexColumnName.substring(pos+1));
         } catch (ColumnNotFoundException e) {
             throw new IllegalArgumentException("Could not find column \"" +  indexColumnName.substring(pos+1) + "\" in index column name of \"" + indexColumnName + "\"", e);
         }
@@ -219,10 +229,11 @@ public class IndexUtil {
 
     private static boolean isEmptyKeyValue(PTable table, ColumnReference ref) {
         byte[] emptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(table);
+        byte[] emptyKeyValueQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
         return (Bytes.compareTo(emptyKeyValueCF, 0, emptyKeyValueCF.length, ref.getFamilyWritable()
                 .get(), ref.getFamilyWritable().getOffset(), ref.getFamilyWritable().getLength()) == 0 && Bytes
-                .compareTo(QueryConstants.EMPTY_COLUMN_BYTES, 0,
-                    QueryConstants.EMPTY_COLUMN_BYTES.length, ref.getQualifierWritable().get(), ref
+                .compareTo(emptyKeyValueQualifier, 0,
+                        emptyKeyValueQualifier.length, ref.getQualifierWritable().get(), ref
                             .getQualifierWritable().getOffset(), ref.getQualifierWritable()
                             .getLength()) == 0);
     }
@@ -254,10 +265,10 @@ public class IndexUtil {
     }
     
     public static List<Mutation> generateIndexData(final PTable table, PTable index,
-            List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
+            final Map<ImmutableBytesPtr, RowMutationState> valuesMap, List<Mutation> dataMutations, final KeyValueBuilder kvBuilder, PhoenixConnection connection)
             throws SQLException {
         try {
-        	final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+        	final ImmutableBytesPtr ptr = new ImmutableBytesPtr();
             IndexMaintainer maintainer = index.getIndexMaintainer(table, connection);
             List<Mutation> indexMutations = Lists.newArrayListWithExpectedSize(dataMutations.size());
             for (final Mutation dataMutation : dataMutations) {
@@ -270,8 +281,6 @@ public class IndexUtil {
                  * updating an existing row.
                  */
                 if (dataMutation instanceof Put) {
-                    // TODO: is this more efficient than looking in our mutation map
-                    // using the key plus finding the PColumn?
                     ValueGetter valueGetter = new ValueGetter() {
                     	
                     	@Override
@@ -286,13 +295,13 @@ public class IndexUtil {
                             if (isEmptyKeyValue(table, ref)) {
                                 return null;
                             }
-                            Map<byte [], List<Cell>> familyMap = dataMutation.getFamilyCellMap();
                             byte[] family = ref.getFamily();
+                            byte[] qualifier = ref.getQualifier();
+                            Map<byte [], List<Cell>> familyMap = dataMutation.getFamilyCellMap();
                             List<Cell> kvs = familyMap.get(family);
                             if (kvs == null) {
                                 return null;
                             }
-                            byte[] qualifier = ref.getQualifier();
                             for (Cell kv : kvs) {
                                 if (Bytes.compareTo(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), family, 0, family.length) == 0 &&
                                     Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), qualifier, 0, qualifier.length) == 0) {
@@ -426,13 +435,18 @@ public class IndexUtil {
     public static TupleProjector getTupleProjector(Scan scan, ColumnReference[] dataColumns) {
         if (dataColumns != null && dataColumns.length != 0) {
             KeyValueSchema keyValueSchema = deserializeLocalIndexJoinSchemaFromScan(scan); 
-            KeyValueColumnExpression[] keyValueColumns = new KeyValueColumnExpression[dataColumns.length];
+            boolean storeColsInSingleCell = scan.getAttribute(BaseScannerRegionObserver.COLUMNS_STORED_IN_SINGLE_CELL)!=null;
+            Expression[] colExpressions = storeColsInSingleCell ? new ArrayColumnExpression[dataColumns.length] : new KeyValueColumnExpression[dataColumns.length];
             for (int i = 0; i < dataColumns.length; i++) {
-                ColumnReference dataColumn = dataColumns[i];
-                KeyValueColumnExpression dataColumnExpr = new KeyValueColumnExpression(keyValueSchema.getField(i), dataColumn.getFamily(), dataColumn.getQualifier());
-                keyValueColumns[i] = dataColumnExpr;
+                byte[] family = dataColumns[i].getFamily();
+                byte[] qualifier = dataColumns[i].getQualifier();
+                Field field = keyValueSchema.getField(i);
+                Expression dataColumnExpr =
+                        storeColsInSingleCell ? new ArrayColumnExpression(field, family, getEncodedColumnQualifier(qualifier)) 
+                            : new KeyValueColumnExpression(field, family, qualifier);
+                colExpressions[i] = dataColumnExpr;
             }
-            return new TupleProjector(keyValueSchema, keyValueColumns);
+            return new TupleProjector(keyValueSchema, colExpressions);
         }
         return null;
     }
@@ -481,8 +495,14 @@ public class IndexUtil {
             ptr.set(indexRowKey, firstCell.getRowOffset() + offset, firstCell.getRowLength() - offset);
             byte[] dataRowKey = indexMaintainer.buildDataRowKey(ptr, viewConstants);
             Get get = new Get(dataRowKey);
+            StorageScheme storageScheme = indexMaintainer.getIndexStorageScheme();
+            boolean colsStoredInSingleCell = storageScheme == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL;
             for (int i = 0; i < dataColumns.length; i++) {
-                get.addColumn(dataColumns[i].getFamily(), dataColumns[i].getQualifier());
+                if (colsStoredInSingleCell) {
+                    get.addFamily(dataColumns[i].getFamily());
+                } else {
+                    get.addColumn(dataColumns[i].getFamily(), dataColumns[i].getQualifier());
+                }
             }
             Result joinResult = null;
             if (dataRegion != null) {
@@ -499,7 +519,8 @@ public class IndexUtil {
                     if (table != null) table.close();
                 }
             }
-            
+            // at this point join result has data from the data table. We now need to take this result and
+            // add it to the cells that we are returning. 
             // TODO: handle null case (but shouldn't happen)
             Tuple joinTuple = new ResultTuple(joinResult);
             // This will create a byte[] that captures all of the values from the data table
@@ -507,12 +528,14 @@ public class IndexUtil {
                     tupleProjector.getSchema().toBytes(joinTuple, tupleProjector.getExpressions(),
                         tupleProjector.getValueBitSet(), ptr);
             KeyValue keyValue =
-                    KeyValueUtil.newKeyValue(firstCell.getRowArray(),firstCell.getRowOffset(),firstCell.getRowLength(), TupleProjector.VALUE_COLUMN_FAMILY,
-                        TupleProjector.VALUE_COLUMN_QUALIFIER, firstCell.getTimestamp(), value, 0, value.length);
+                    KeyValueUtil.newKeyValue(firstCell.getRowArray(),firstCell.getRowOffset(),firstCell.getRowLength(), VALUE_COLUMN_FAMILY,
+                        VALUE_COLUMN_QUALIFIER, firstCell.getTimestamp(), value, 0, value.length);
             result.add(keyValue);
         }
-        for (int i = 0; i < result.size(); i++) {
-            final Cell cell = result.get(i);
+        
+        ListIterator<Cell> itr = result.listIterator();
+        while (itr.hasNext()) {
+            final Cell cell = itr.next();
             // TODO: Create DelegateCell class instead
             Cell newCell = new Cell() {
 
@@ -528,7 +551,7 @@ public class IndexUtil {
 
                 @Override
                 public short getRowLength() {
-                    return (short)(cell.getRowLength() - offset);
+                    return (short) (cell.getRowLength() - offset);
                 }
 
                 @Override
@@ -632,8 +655,7 @@ public class IndexUtil {
                     return cell.getTagsLengthUnsigned();
                 }
             };
-            // Wrap cell in cell that offsets row key
-            result.set(i, newCell);
+            itr.set(newCell);
         }
     }
     
@@ -686,4 +708,9 @@ public class IndexUtil {
         }
         return true;
     }
+    
+    public static boolean isLocalIndexFamily(String family) {
+        return family.indexOf(LOCAL_INDEX_COLUMN_FAMILY_PREFIX) != -1;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
index b8850d2..071de66 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/KeyValueUtil.java
@@ -25,10 +25,8 @@ import java.util.List;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.KeyValue.KVComparator;
 import org.apache.hadoop.hbase.KeyValue.Type;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
-import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 
 /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 01e1575..e7e91cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -643,4 +643,10 @@ public class MetaDataUtil {
     public static boolean isLocalIndexFamily(byte[] cf) {
         return Bytes.startsWith(cf, QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX_BYTES);
     }
+    
+    public static final byte[] getPhysicalTableRowForView(PTable view) {
+        byte[] physicalTableSchemaName = Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(view.getPhysicalName().getString()));
+        byte[] physicalTableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(view.getPhysicalName().getString()));
+        return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index 0c74b84..a946575 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -495,9 +495,9 @@ public class PhoenixRuntime {
             String familyName = tokens[0];
             String familyColumn = tokens[1];
             PColumnFamily family = table.getColumnFamily(familyName);
-            pColumn = family.getColumn(familyColumn);
+            pColumn = family.getPColumnForColumnName(familyColumn);
         } else {
-            pColumn = table.getColumn(columnName);
+            pColumn = table.getPColumnForColumnName(columnName);
         }
         return getColumnInfo(pColumn);
     }
@@ -1188,9 +1188,9 @@ public class PhoenixRuntime {
         PColumn pColumn = null;
         if (familyName != null) {
             PColumnFamily family = table.getColumnFamily(familyName);
-            pColumn = family.getColumn(columnName);
+            pColumn = family.getPColumnForColumnName(columnName);
         } else {
-            pColumn = table.getColumn(columnName);
+            pColumn = table.getPColumnForColumnName(columnName);
         }
         return pColumn;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/ResultUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ResultUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ResultUtil.java
index dba6550..f97230b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ResultUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ResultUtil.java
@@ -18,7 +18,6 @@
 package org.apache.phoenix.util;
 
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 
@@ -129,63 +128,4 @@ public class ResultUtil {
         return Bytes.compareTo(getRawBytes(r1), getKeyOffset(r1), getKeyLength(r1), getRawBytes(r2), getKeyOffset(r2), getKeyLength(r2));
     }
 
-    /**
-     * Binary search for latest column value without allocating memory in the process
-     */
-    public static KeyValue getColumnLatest(Result r, byte[] family, byte[] qualifier) {
-        byte[] rbytes = getRawBytes(r);
-        int roffset = getKeyOffset(r);
-        int rlength = getKeyLength(r);
-        return getColumnLatest(r, rbytes, roffset, rlength, family, 0, family.length, qualifier, 0, qualifier.length);
-    }
-
-    public static KeyValue getSearchTerm(Result r, byte[] family, byte[] qualifier) {
-        byte[] rbytes = getRawBytes(r);
-        int roffset = getKeyOffset(r);
-        int rlength = getKeyLength(r);
-        return KeyValue.createFirstOnRow(rbytes, roffset, rlength, family, 0, family.length, qualifier, 0, qualifier.length);
-    }
-    /**
-     * Binary search for latest column value without allocating memory in the process
-     */
-    public static KeyValue getColumnLatest(Result r, byte[] row, int roffset, int rlength, byte[] family, int foffset, int flength, byte[] qualifier, int qoffset, int qlength) {
-        KeyValue searchTerm = KeyValue.createFirstOnRow(row, roffset, rlength, family, foffset, flength, qualifier, qoffset, qlength);
-        return getColumnLatest(r,searchTerm);
-        
-    }
-
-     /**
-     * Binary search for latest column value without allocating memory in the process
-     * @param r
-     * @param searchTerm
-     */
-    @SuppressWarnings("deprecation")
-    public static KeyValue getColumnLatest(Result r, KeyValue searchTerm) {
-        KeyValue [] kvs = r.raw(); // side effect possibly.
-        if (kvs == null || kvs.length == 0) {
-          return null;
-        }
-        
-        // pos === ( -(insertion point) - 1)
-        int pos = Arrays.binarySearch(kvs, searchTerm, KeyValue.COMPARATOR);
-        // never will exact match
-        if (pos < 0) {
-          pos = (pos+1) * -1;
-          // pos is now insertion point
-        }
-        if (pos == kvs.length) {
-          return null; // doesn't exist
-        }
-
-        KeyValue kv = kvs[pos];
-        if (Bytes.compareTo(kv.getBuffer(), kv.getFamilyOffset(), kv.getFamilyLength(),
-                searchTerm.getBuffer(), searchTerm.getFamilyOffset(), searchTerm.getFamilyLength()) != 0) {
-            return null;
-        }
-        if (Bytes.compareTo(kv.getBuffer(), kv.getQualifierOffset(), kv.getQualifierLength(),
-                searchTerm.getBuffer(), searchTerm.getQualifierOffset(), searchTerm.getQualifierLength()) != 0) {
-            return null;
-        }
-        return kv;
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index acaeb31..8c2f58d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
@@ -23,6 +23,7 @@ import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.CUSTOM_AN
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_ACTUAL_START_ROW;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_START_ROW_SUFFIX;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_STOP_ROW_SUFFIX;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
 
 import java.io.IOException;
 import java.sql.SQLException;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.TimeRange;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.compile.ScanRanges;
@@ -63,13 +65,16 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.IllegalDataException;
+import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PVarbinary;
 
 import com.google.common.collect.Iterators;
@@ -894,6 +899,40 @@ public class ScanUtil {
         }
         return true;
     }
+    
+    public static Pair<Integer, Integer> getMinMaxQualifiersFromScan(Scan scan) {
+        Integer minQ = null, maxQ = null;
+        byte[] minQualifier = scan.getAttribute(BaseScannerRegionObserver.MIN_QUALIFIER);
+        if (minQualifier != null) {
+            minQ = getEncodedColumnQualifier(minQualifier);
+        }
+        byte[] maxQualifier = scan.getAttribute(BaseScannerRegionObserver.MAX_QUALIFIER);
+        if (maxQualifier != null) {
+            maxQ = getEncodedColumnQualifier(maxQualifier);
+        }
+        if (minQualifier == null) {
+            return null;
+        }
+        return new Pair<>(minQ, maxQ);
+    }
+    
+    public static boolean useQualifierAsIndex(Pair<Integer, Integer> minMaxQualifiers) {
+        return minMaxQualifiers != null;
+    }
+    
+    public static boolean setQualifierRanges(PTable table) {
+        return table.getStorageScheme() != null && table.getStorageScheme() == StorageScheme.ENCODED_COLUMN_NAMES
+        		&& !table.isTransactional() && !hasDynamicColumns(table);
+    }
+    
+    public static boolean hasDynamicColumns(PTable table) {
+        for (PColumn col : table.getColumns()) {
+            if (col.isDynamic()) {
+                return true;
+            }
+        }
+        return false;
+    }
 
     public static boolean isIndexRebuild(Scan scan) {
         return scan.getAttribute((BaseScannerRegionObserver.REBUILD_INDEXES)) != null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 003fd73..3918545 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -47,6 +47,8 @@ import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.exception.DataExceedsCapacityException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
@@ -69,6 +71,7 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SortOrder;
@@ -155,8 +158,9 @@ public class SchemaUtil {
     			rowSize += KeyValue.getKeyValueDataStructureSize(keyLength, column.getFamilyName().getBytes().length, column.getName().getBytes().length, valueLength);
     		}
     	}
+    	byte[] emptyKeyValueKV = EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
     	// Empty key value
-    	rowSize += KeyValue.getKeyValueDataStructureSize(keyLength, getEmptyColumnFamily(table).length, QueryConstants.EMPTY_COLUMN_BYTES.length, 0);
+    	rowSize += KeyValue.getKeyValueDataStructureSize(keyLength, getEmptyColumnFamily(table).length, emptyKeyValueKV.length, 0);
     	return rowSize;
     }
     
@@ -410,7 +414,7 @@ public class SchemaUtil {
                 }
             } else {
                 try {
-                    return table.getColumnFamily(familyName.getString()).getColumn(column.getName().getString()).getName().getString();
+                    return table.getColumnFamily(familyName.getString()).getPColumnForColumnName(column.getName().getString()).getName().getString();
                 } catch (ColumnFamilyNotFoundException e) {
                     continue; // Shouldn't happen
                 } catch (ColumnNotFoundException e) {
@@ -615,7 +619,7 @@ public class SchemaUtil {
     
     public static boolean columnExists(PTable table, String columnName) {
         try {
-            table.getColumn(columnName);
+            table.getPColumnForColumnName(columnName);
             return true;
         } catch (ColumnNotFoundException e) {
             return false;
@@ -1044,7 +1048,7 @@ public class SchemaUtil {
         }
         return getStrippedName(physicalTableName, indexPrefix);
     }
-
+    
     private static String getStrippedName(String physicalTableName, String indexPrefix) {
         return physicalTableName.indexOf(indexPrefix) == 0 ? physicalTableName.substring(indexPrefix.length())
                 : physicalTableName;
@@ -1065,4 +1069,46 @@ public class SchemaUtil {
             return normalizeIdentifier(tableName);
         }
     }
+
+    /**
+     * Pads the data in ptr by the required amount for fixed width data types
+     */
+    public static void padData(String tableName, PColumn column, ImmutableBytesWritable ptr) {
+        PDataType type = column.getDataType();
+        byte[] byteValue = ptr.get();
+        boolean isNull = type.isNull(byteValue);
+        Integer maxLength = column.getMaxLength();
+        if (!isNull && type.isFixedWidth() && maxLength != null) {
+            if (ptr.getLength() < maxLength) {
+                type.pad(ptr, maxLength, column.getSortOrder());
+            } else if (ptr.getLength() > maxLength) {
+                throw new DataExceedsCapacityException(tableName + "." + column.getName().getString() + " may not exceed " + maxLength + " bytes (" + type.toObject(byteValue) + ")");
+            }
+        }
+    }
+    
+    public static Map<String, Pair<Integer, Integer>> getQualifierRanges(PTable table) {
+        Preconditions.checkArgument(table.getStorageScheme() == StorageScheme.ENCODED_COLUMN_NAMES,
+            "Use this method only for tables with storage scheme "
+                    + StorageScheme.ENCODED_COLUMN_NAMES.name());
+        Map<String, Pair<Integer, Integer>> toReturn = Maps.newHashMapWithExpectedSize(table.getColumns().size());
+        for (PColumn column : table.getColumns()) {
+            if (!isPKColumn(column)) {
+                String colFamily = column.getFamilyName().getString();
+                Pair<Integer, Integer> minMaxQualifiers = toReturn.get(colFamily);
+                Integer encodedColumnQualifier = column.getEncodedColumnQualifier();
+                if (minMaxQualifiers == null) {
+                    minMaxQualifiers = new Pair<>(encodedColumnQualifier, encodedColumnQualifier);
+                    toReturn.put(colFamily, minMaxQualifiers);
+                } else {
+                    if (encodedColumnQualifier < minMaxQualifiers.getFirst()) {
+                        minMaxQualifiers.setFirst(encodedColumnQualifier);
+                    } else if (encodedColumnQualifier > minMaxQualifiers.getSecond()) {
+                        minMaxQualifiers.setSecond(encodedColumnQualifier);
+                    }
+                }
+            }
+        }
+        return toReturn;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/compile/HavingCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/HavingCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/HavingCompilerTest.java
index 1c7477d..ae2bd14 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/HavingCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/HavingCompilerTest.java
@@ -156,7 +156,7 @@ public class HavingCompilerTest extends BaseConnectionlessQueryTest {
         String query = "select count(1) from atable group by a_string having count(1) >= 1 or a_string = 'foo'";
         List<Object> binds = Collections.emptyList();
         Expressions expressions = compileStatement(query,binds);
-        PColumn aCol = ATABLE.getColumn("A_STRING");
+        PColumn aCol = ATABLE.getPColumnForColumnName("A_STRING");
         Expression h = or(
                 constantComparison(CompareOp.GREATER_OR_EQUAL, new CountAggregateFunction(),1L),
                 constantComparison(CompareOp.EQUAL, 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
index 7488c72..ddf1086 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryCompilerTest.java
@@ -170,7 +170,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
             String query = "CREATE TABLE t1 (k integer not null primary key, a.k decimal, b.k decimal)";
             conn.createStatement().execute(query);
             PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-            PColumn c = pconn.getTable(new PTableKey(pconn.getTenantId(), "T1")).getColumn("K");
+            PColumn c = pconn.getTable(new PTableKey(pconn.getTenantId(), "T1")).getPColumnForColumnName("K");
             assertTrue(SchemaUtil.isPKColumn(c));
         } finally {
             conn.close();
@@ -2623,7 +2623,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         conn.createStatement().execute(ddl);
         PTable table = conn.unwrap(PhoenixConnection.class).getMetaDataCache()
                 .getTableRef(new PTableKey(null,"TABLE_WITH_DEFAULT")).getTable();
-        assertNull(table.getColumn("V").getExpressionStr());
+        assertNull(table.getPColumnForColumnName("V").getExpressionStr());
     }
 
     @Test
@@ -2638,7 +2638,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         conn.createStatement().execute(ddl2);
         PTable table = conn.unwrap(PhoenixConnection.class).getMetaDataCache()
                 .getTableRef(new PTableKey(null,"TABLE_WITH_DEFAULT")).getTable();
-        assertNull(table.getColumn("V").getExpressionStr());
+        assertNull(table.getPColumnForColumnName("V").getExpressionStr());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
index c0bff8a..c65408e 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/WhereCompilerTest.java
@@ -52,7 +52,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
@@ -67,8 +66,10 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PVarchar;
@@ -118,9 +119,9 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-        Expression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getColumn("ID").getPosition()).newColumnExpression();
+        Expression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getPColumnForColumnName("ID").getPosition()).newColumnExpression();
         Expression id = new RowKeyColumnExpression(idExpression,new RowKeyValueAccessor(plan.getTableRef().getTable().getPKColumns(),0));
-        Expression company = new KeyValueColumnExpression(plan.getTableRef().getTable().getColumn("COMPANY"));
+        Expression company = new KeyValueColumnExpression(plan.getTableRef().getTable().getPColumnForColumnName("COMPANY"), false);
         // FilterList has no equals implementation
         assertTrue(filter instanceof FilterList);
         FilterList filterList = (FilterList)filter;
@@ -148,11 +149,11 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-        PColumn column = plan.getTableRef().getTable().getColumn("COMPANY");
+        PColumn column = plan.getTableRef().getTable().getPColumnForColumnName("COMPANY");
         assertEquals(
                 singleKVFilter(constantComparison(
                     CompareOp.EQUAL,
-                    new KeyValueColumnExpression(column),
+                    new KeyValueColumnExpression(column, false),
                     "c3")),
                 filter);
     }
@@ -939,16 +940,18 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-
+        PTable table = plan.getTableRef().getTable();
+        Expression aInteger = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+        Expression aString = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_STRING").getPosition()).newColumnExpression();
         assertEquals(
             multiKVFilter(and(
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_INTEGER,
+                    aInteger,
                     0),
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_STRING,
+                    aString,
                     "foo"))),
             filter);
         
@@ -971,16 +974,18 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-
+        PTable table = plan.getTableRef().getTable();
+        Expression aInteger = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+        Expression aString = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_STRING").getPosition()).newColumnExpression();
         assertEquals(
             multiKVFilter(and(
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_INTEGER,
+                    aInteger,
                     0),
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_STRING,
+                    aString,
                     "foo"))),
             filter);
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
index 62aafa5..ca6a471 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/CorrelatePlanTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
@@ -55,12 +56,15 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PName;
 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.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
@@ -247,16 +251,16 @@ public class CorrelatePlanTest {
         for (int i = 0; i < row.length; i++) {
             String name = ParseNodeFactory.createTempAlias();
             Expression expr = LiteralExpression.newConstant(row[i]);
-            columns.add(new PColumnImpl(PNameFactory.newName(name), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY),
+            columns.add(new PColumnImpl(PNameFactory.newName(name), PNameFactory.newName(VALUE_COLUMN_FAMILY),
                     expr.getDataType(), expr.getMaxLength(), expr.getScale(), expr.isNullable(),
-                    i, expr.getSortOrder(), null, null, false, name, false, false));
+                    i, expr.getSortOrder(), null, null, false, name, false, false, null));
         }
         try {
             PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME,
                     PTableType.SUBQUERY, null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                     null, null, columns, null, null, Collections.<PTable>emptyList(),
                     false, Collections.<PName>emptyList(), null, null, false, false, false, null,
-                    null, null, true, false, 0, 0L, Boolean.FALSE, null, false);
+                    null, null, true, false, 0, 0L, Boolean.FALSE, null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, EncodedCQCounter.NULL_COUNTER);
             TableRef sourceTable = new TableRef(pTable);
             List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
index 1b16d40..7fff2c4 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/LiteralResultIteratorPlanTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
@@ -49,15 +50,18 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.SelectStatement;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.EncodedCQCounter;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.junit.Test;
@@ -170,14 +174,14 @@ public class LiteralResultIteratorPlanTest {
             String name = ParseNodeFactory.createTempAlias();
             Expression expr = LiteralExpression.newConstant(row[i]);
             columns.add(new PColumnImpl(PNameFactory.newName(name),
-                    PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), expr.getDataType(), expr.getMaxLength(),
-                    expr.getScale(), expr.isNullable(), i, expr.getSortOrder(), null, null, false, name, false, false));
+                    PNameFactory.newName(VALUE_COLUMN_FAMILY), expr.getDataType(), expr.getMaxLength(),
+                    expr.getScale(), expr.isNullable(), i, expr.getSortOrder(), null, null, false, name, false, false, null));
         }
         try {
             PTable pTable = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
                     MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
                     Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
-                    false, null, null, null, true, false, 0, 0L, false, null, false);
+                    false, null, null, null, true, false, 0, 0L, false, null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, EncodedCQCounter.NULL_COUNTER);
             TableRef sourceTable = new TableRef(pTable);
             List<ColumnRef> sourceColumnRefs = Lists.<ColumnRef> newArrayList();
             for (PColumn column : sourceTable.getTable().getColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
index 50e2721..ff62f63 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/UnnestArrayPlanTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 import static org.apache.phoenix.util.PhoenixRuntime.CONNECTIONLESS;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL;
 import static org.apache.phoenix.util.PhoenixRuntime.JDBC_PROTOCOL_SEPARATOR;
@@ -117,8 +118,8 @@ public class UnnestArrayPlanTest {
         LiteralExpression dummy = LiteralExpression.newConstant(null, arrayType);
         RowKeyValueAccessor accessor = new RowKeyValueAccessor(Arrays.asList(dummy), 0);
         UnnestArrayPlan plan = new UnnestArrayPlan(subPlan, new RowKeyColumnExpression(dummy, accessor), withOrdinality);
-        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false);
-        PColumn indexColumn = withOrdinality ? new PColumnImpl(PNameFactory.newName("IDX"), PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false) : null;
+        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, null);
+        PColumn indexColumn = withOrdinality ? new PColumnImpl(PNameFactory.newName("IDX"), PNameFactory.newName(VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, null) : null;
         List<PColumn> columns = withOrdinality ? Arrays.asList(elemColumn, indexColumn) : Arrays.asList(elemColumn);
         ProjectedColumnExpression elemExpr = new ProjectedColumnExpression(elemColumn, columns, 0, elemColumn.getName().getString());
         ProjectedColumnExpression indexExpr = withOrdinality ? new ProjectedColumnExpression(indexColumn, columns, 1, indexColumn.getName().getString()) : null;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
index 7ee579c..98c2495 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/ColumnExpressionTest.java
@@ -41,8 +41,8 @@ public class ColumnExpressionTest {
         int maxLen = 30;
         int scale = 5;
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PDecimal.INSTANCE, maxLen, scale,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false);
-        ColumnExpression colExp = new KeyValueColumnExpression(column);
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, 0);
+        ColumnExpression colExp = new KeyValueColumnExpression(column, true);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
         colExp.write(dOut);
@@ -61,8 +61,8 @@ public class ColumnExpressionTest {
     public void testSerializationWithNullScale() throws Exception {
         int maxLen = 30;
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PBinary.INSTANCE, maxLen, null,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false);
-        ColumnExpression colExp = new KeyValueColumnExpression(column);
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, 0);
+        ColumnExpression colExp = new KeyValueColumnExpression(column, true);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
         colExp.write(dOut);
@@ -81,8 +81,8 @@ public class ColumnExpressionTest {
     public void testSerializationWithNullMaxLength() throws Exception {
         int scale = 5;
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PVarchar.INSTANCE, null, scale,
-                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false);
-        ColumnExpression colExp = new KeyValueColumnExpression(column);
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, 0);
+        ColumnExpression colExp = new KeyValueColumnExpression(column, true);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
         colExp.write(dOut);
@@ -100,8 +100,8 @@ public class ColumnExpressionTest {
     @Test
     public void testSerializationWithNullScaleAndMaxLength() throws Exception {
         PColumn column = new PColumnImpl(PNameFactory.newName("c1"), PNameFactory.newName("f1"), PDecimal.INSTANCE, null, null, true,
-                20, SortOrder.getDefault(), 0, null, false, null, false, false);
-        ColumnExpression colExp = new KeyValueColumnExpression(column);
+                20, SortOrder.getDefault(), 0, null, false, null, false, false, 0);
+        ColumnExpression colExp = new KeyValueColumnExpression(column, true);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
         colExp.write(dOut);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
index 112109e..5887e5b 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/index/IndexMaintainerTest.java
@@ -135,8 +135,7 @@ public class IndexMaintainerTest  extends BaseConnectionlessQueryTest {
             }
             ValueGetter valueGetter = newValueGetter(row, valueMap);
             
-            List<Mutation> indexMutations =
-                    IndexTestUtil.generateIndexData(index, table, dataMutation, ptr, builder);
+            List<Mutation> indexMutations = IndexTestUtil.generateIndexData(index, table, dataMutation, ptr, builder);
             assertEquals(1,indexMutations.size());
             assertTrue(indexMutations.get(0) instanceof Put);
             Mutation indexMutation = indexMutations.get(0);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
index 791eb23..ddd8241 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/iterate/AggregateResultScannerTest.java
@@ -74,4 +74,4 @@ public class AggregateResultScannerTest extends BaseConnectionlessQueryTest {
         ResultIterator scanner = new GroupedAggregatingResultIterator(new MergeSortRowKeyResultIterator(iterators), aggregators);
         AssertResults.assertResults(scanner, expectedResults);
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
index b74cefb..750e46f 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseConnectionlessQueryTest.java
@@ -119,15 +119,15 @@ public class BaseConnectionlessQueryTest extends BaseTest {
         try {
             PTable table = conn.getTable(new PTableKey(null, ATABLE_NAME));
             ATABLE = table;
-            ORGANIZATION_ID = new ColumnRef(new TableRef(table), table.getColumn("ORGANIZATION_ID").getPosition()).newColumnExpression();
-            ENTITY_ID = new ColumnRef(new TableRef(table), table.getColumn("ENTITY_ID").getPosition()).newColumnExpression();
-            A_INTEGER = new ColumnRef(new TableRef(table), table.getColumn("A_INTEGER").getPosition()).newColumnExpression();
-            A_STRING = new ColumnRef(new TableRef(table), table.getColumn("A_STRING").getPosition()).newColumnExpression();
-            B_STRING = new ColumnRef(new TableRef(table), table.getColumn("B_STRING").getPosition()).newColumnExpression();
-            A_DATE = new ColumnRef(new TableRef(table), table.getColumn("A_DATE").getPosition()).newColumnExpression();
-            A_TIME = new ColumnRef(new TableRef(table), table.getColumn("A_TIME").getPosition()).newColumnExpression();
-            A_TIMESTAMP = new ColumnRef(new TableRef(table), table.getColumn("A_TIMESTAMP").getPosition()).newColumnExpression();
-            X_DECIMAL = new ColumnRef(new TableRef(table), table.getColumn("X_DECIMAL").getPosition()).newColumnExpression();
+            ORGANIZATION_ID = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("ORGANIZATION_ID").getPosition()).newColumnExpression();
+            ENTITY_ID = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("ENTITY_ID").getPosition()).newColumnExpression();
+            A_INTEGER = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+            A_STRING = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_STRING").getPosition()).newColumnExpression();
+            B_STRING = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("B_STRING").getPosition()).newColumnExpression();
+            A_DATE = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_DATE").getPosition()).newColumnExpression();
+            A_TIME = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_TIME").getPosition()).newColumnExpression();
+            A_TIMESTAMP = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_TIMESTAMP").getPosition()).newColumnExpression();
+            X_DECIMAL = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("X_DECIMAL").getPosition()).newColumnExpression();
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
index 3396cf8..089c5f1 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/ConnectionlessTest.java
@@ -136,7 +136,6 @@ public class ConnectionlessTest {
         conn.rollback(); // to clear the list of mutations for the next
     }
     
-    @SuppressWarnings("deprecation")
     private static void assertRow1(Iterator<KeyValue> iterator, byte[] expectedRowKey1) {
         KeyValue kv;
         assertTrue(iterator.hasNext());
@@ -153,7 +152,6 @@ public class ConnectionlessTest {
         assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue()));
     }
 
-    @SuppressWarnings("deprecation")
     private static void assertRow2(Iterator<KeyValue> iterator, byte[] expectedRowKey2) {
         KeyValue kv;
         assertTrue(iterator.hasNext());


[13/13] phoenix git commit: Fix compilation failure

Posted by sa...@apache.org.
Fix compilation failure


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

Branch: refs/heads/encodecolumns2
Commit: 045ffcbe2956e4f8302d7a108db142a39d945cc1
Parents: 56c1767
Author: Samarth <sa...@salesforce.com>
Authored: Mon Nov 7 13:27:06 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Nov 7 23:54:24 2016 -0800

----------------------------------------------------------------------
 .../src/main/java/org/apache/phoenix/util/PhoenixRuntime.java    | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/045ffcbe/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
index a946575..563ccab 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/PhoenixRuntime.java
@@ -1148,9 +1148,9 @@ public class PhoenixRuntime {
         PColumn pColumn = null;
         if (familyName != null) {
             PColumnFamily family = table.getColumnFamily(familyName);
-            pColumn = family.getColumn(columnName);
+            pColumn = family.getPColumnForColumnName(columnName);
         } else {
-            pColumn = table.getColumn(columnName);
+            pColumn = table.getPColumnForColumnName(columnName);
         }
         return pColumn;
     }


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 b8b8b2f..2f0c00b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -269,6 +269,16 @@ public final class PTableProtos {
      * <code>optional bool isDynamic = 14;</code>
      */
     boolean getIsDynamic();
+
+    // optional int32 columnQualifier = 15;
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    boolean hasColumnQualifier();
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    int getColumnQualifier();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -391,6 +401,11 @@ public final class PTableProtos {
               isDynamic_ = input.readBool();
               break;
             }
+            case 120: {
+              bitField0_ |= 0x00004000;
+              columnQualifier_ = input.readInt32();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -709,6 +724,22 @@ public final class PTableProtos {
       return isDynamic_;
     }
 
+    // optional int32 columnQualifier = 15;
+    public static final int COLUMNQUALIFIER_FIELD_NUMBER = 15;
+    private int columnQualifier_;
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    public boolean hasColumnQualifier() {
+      return ((bitField0_ & 0x00004000) == 0x00004000);
+    }
+    /**
+     * <code>optional int32 columnQualifier = 15;</code>
+     */
+    public int getColumnQualifier() {
+      return columnQualifier_;
+    }
+
     private void initFields() {
       columnNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       familyNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -724,6 +755,7 @@ public final class PTableProtos {
       expression_ = "";
       isRowTimestamp_ = false;
       isDynamic_ = false;
+      columnQualifier_ = 0;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -799,6 +831,9 @@ public final class PTableProtos {
       if (((bitField0_ & 0x00002000) == 0x00002000)) {
         output.writeBool(14, isDynamic_);
       }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        output.writeInt32(15, columnQualifier_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -864,6 +899,10 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(14, isDynamic_);
       }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(15, columnQualifier_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -957,6 +996,11 @@ public final class PTableProtos {
         result = result && (getIsDynamic()
             == other.getIsDynamic());
       }
+      result = result && (hasColumnQualifier() == other.hasColumnQualifier());
+      if (hasColumnQualifier()) {
+        result = result && (getColumnQualifier()
+            == other.getColumnQualifier());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -1026,6 +1070,10 @@ public final class PTableProtos {
         hash = (37 * hash) + ISDYNAMIC_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getIsDynamic());
       }
+      if (hasColumnQualifier()) {
+        hash = (37 * hash) + COLUMNQUALIFIER_FIELD_NUMBER;
+        hash = (53 * hash) + getColumnQualifier();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -1163,6 +1211,8 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x00001000);
         isDynamic_ = false;
         bitField0_ = (bitField0_ & ~0x00002000);
+        columnQualifier_ = 0;
+        bitField0_ = (bitField0_ & ~0x00004000);
         return this;
       }
 
@@ -1247,6 +1297,10 @@ public final class PTableProtos {
           to_bitField0_ |= 0x00002000;
         }
         result.isDynamic_ = isDynamic_;
+        if (((from_bitField0_ & 0x00004000) == 0x00004000)) {
+          to_bitField0_ |= 0x00004000;
+        }
+        result.columnQualifier_ = columnQualifier_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1309,6 +1363,9 @@ public final class PTableProtos {
         if (other.hasIsDynamic()) {
           setIsDynamic(other.getIsDynamic());
         }
+        if (other.hasColumnQualifier()) {
+          setColumnQualifier(other.getColumnQualifier());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1909,6 +1966,39 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional int32 columnQualifier = 15;
+      private int columnQualifier_ ;
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public boolean hasColumnQualifier() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public int getColumnQualifier() {
+        return columnQualifier_;
+      }
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public Builder setColumnQualifier(int value) {
+        bitField0_ |= 0x00004000;
+        columnQualifier_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional int32 columnQualifier = 15;</code>
+       */
+      public Builder clearColumnQualifier() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        columnQualifier_ = 0;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PColumn)
     }
 
@@ -3372,6 +3462,41 @@ public final class PTableProtos {
      * <code>optional bytes parentNameBytes = 33;</code>
      */
     com.google.protobuf.ByteString getParentNameBytes();
+
+    // optional bytes storageScheme = 34;
+    /**
+     * <code>optional bytes storageScheme = 34;</code>
+     */
+    boolean hasStorageScheme();
+    /**
+     * <code>optional bytes storageScheme = 34;</code>
+     */
+    com.google.protobuf.ByteString getStorageScheme();
+
+    // repeated .EncodedCQCounter encodedCQCounters = 35;
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> 
+        getEncodedCQCountersList();
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index);
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    int getEncodedCQCountersCount();
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
+        getEncodedCQCountersOrBuilderList();
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
+        int index);
   }
   /**
    * Protobuf type {@code PTable}
@@ -3407,6 +3532,7 @@ public final class PTableProtos {
         throws com.google.protobuf.InvalidProtocolBufferException {
       initFields();
       int mutable_bitField0_ = 0;
+      int mutable_bitField1_ = 0;
       com.google.protobuf.UnknownFieldSet.Builder unknownFields =
           com.google.protobuf.UnknownFieldSet.newBuilder();
       try {
@@ -3599,6 +3725,19 @@ public final class PTableProtos {
               parentNameBytes_ = input.readBytes();
               break;
             }
+            case 274: {
+              bitField0_ |= 0x20000000;
+              storageScheme_ = input.readBytes();
+              break;
+            }
+            case 282: {
+              if (!((mutable_bitField1_ & 0x00000002) == 0x00000002)) {
+                encodedCQCounters_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter>();
+                mutable_bitField1_ |= 0x00000002;
+              }
+              encodedCQCounters_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.PARSER, extensionRegistry));
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3616,6 +3755,9 @@ public final class PTableProtos {
         if (((mutable_bitField0_ & 0x00020000) == 0x00020000)) {
           physicalNames_ = java.util.Collections.unmodifiableList(physicalNames_);
         }
+        if (((mutable_bitField1_ & 0x00000002) == 0x00000002)) {
+          encodedCQCounters_ = java.util.Collections.unmodifiableList(encodedCQCounters_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -4275,6 +4417,58 @@ public final class PTableProtos {
       return parentNameBytes_;
     }
 
+    // optional bytes storageScheme = 34;
+    public static final int STORAGESCHEME_FIELD_NUMBER = 34;
+    private com.google.protobuf.ByteString storageScheme_;
+    /**
+     * <code>optional bytes storageScheme = 34;</code>
+     */
+    public boolean hasStorageScheme() {
+      return ((bitField0_ & 0x20000000) == 0x20000000);
+    }
+    /**
+     * <code>optional bytes storageScheme = 34;</code>
+     */
+    public com.google.protobuf.ByteString getStorageScheme() {
+      return storageScheme_;
+    }
+
+    // repeated .EncodedCQCounter encodedCQCounters = 35;
+    public static final int ENCODEDCQCOUNTERS_FIELD_NUMBER = 35;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> encodedCQCounters_;
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> getEncodedCQCountersList() {
+      return encodedCQCounters_;
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
+        getEncodedCQCountersOrBuilderList() {
+      return encodedCQCounters_;
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    public int getEncodedCQCountersCount() {
+      return encodedCQCounters_.size();
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index) {
+      return encodedCQCounters_.get(index);
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
+        int index) {
+      return encodedCQCounters_.get(index);
+    }
+
     private void initFields() {
       schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       tableNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -4308,6 +4502,8 @@ public final class PTableProtos {
       autoParititonSeqName_ = "";
       isAppendOnlySchema_ = false;
       parentNameBytes_ = com.google.protobuf.ByteString.EMPTY;
+      storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+      encodedCQCounters_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4362,6 +4558,12 @@ public final class PTableProtos {
           return false;
         }
       }
+      for (int i = 0; i < getEncodedCQCountersCount(); i++) {
+        if (!getEncodedCQCounters(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -4465,6 +4667,12 @@ public final class PTableProtos {
       if (((bitField0_ & 0x10000000) == 0x10000000)) {
         output.writeBytes(33, parentNameBytes_);
       }
+      if (((bitField0_ & 0x20000000) == 0x20000000)) {
+        output.writeBytes(34, storageScheme_);
+      }
+      for (int i = 0; i < encodedCQCounters_.size(); i++) {
+        output.writeMessage(35, encodedCQCounters_.get(i));
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4607,6 +4815,14 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(33, parentNameBytes_);
       }
+      if (((bitField0_ & 0x20000000) == 0x20000000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(34, storageScheme_);
+      }
+      for (int i = 0; i < encodedCQCounters_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(35, encodedCQCounters_.get(i));
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4781,6 +4997,13 @@ public final class PTableProtos {
         result = result && getParentNameBytes()
             .equals(other.getParentNameBytes());
       }
+      result = result && (hasStorageScheme() == other.hasStorageScheme());
+      if (hasStorageScheme()) {
+        result = result && getStorageScheme()
+            .equals(other.getStorageScheme());
+      }
+      result = result && getEncodedCQCountersList()
+          .equals(other.getEncodedCQCountersList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4922,6 +5145,14 @@ public final class PTableProtos {
         hash = (37 * hash) + PARENTNAMEBYTES_FIELD_NUMBER;
         hash = (53 * hash) + getParentNameBytes().hashCode();
       }
+      if (hasStorageScheme()) {
+        hash = (37 * hash) + STORAGESCHEME_FIELD_NUMBER;
+        hash = (53 * hash) + getStorageScheme().hashCode();
+      }
+      if (getEncodedCQCountersCount() > 0) {
+        hash = (37 * hash) + ENCODEDCQCOUNTERS_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodedCQCountersList().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5025,6 +5256,7 @@ public final class PTableProtos {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getColumnsFieldBuilder();
           getIndexesFieldBuilder();
+          getEncodedCQCountersFieldBuilder();
         }
       }
       private static Builder create() {
@@ -5105,6 +5337,14 @@ public final class PTableProtos {
         bitField0_ = (bitField0_ & ~0x40000000);
         parentNameBytes_ = com.google.protobuf.ByteString.EMPTY;
         bitField0_ = (bitField0_ & ~0x80000000);
+        storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+        bitField1_ = (bitField1_ & ~0x00000001);
+        if (encodedCQCountersBuilder_ == null) {
+          encodedCQCounters_ = java.util.Collections.emptyList();
+          bitField1_ = (bitField1_ & ~0x00000002);
+        } else {
+          encodedCQCountersBuilder_.clear();
+        }
         return this;
       }
 
@@ -5132,6 +5372,7 @@ public final class PTableProtos {
       public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable buildPartial() {
         org.apache.phoenix.coprocessor.generated.PTableProtos.PTable result = new org.apache.phoenix.coprocessor.generated.PTableProtos.PTable(this);
         int from_bitField0_ = bitField0_;
+        int from_bitField1_ = bitField1_;
         int to_bitField0_ = 0;
         if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
           to_bitField0_ |= 0x00000001;
@@ -5272,6 +5513,19 @@ public final class PTableProtos {
           to_bitField0_ |= 0x10000000;
         }
         result.parentNameBytes_ = parentNameBytes_;
+        if (((from_bitField1_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x20000000;
+        }
+        result.storageScheme_ = storageScheme_;
+        if (encodedCQCountersBuilder_ == null) {
+          if (((bitField1_ & 0x00000002) == 0x00000002)) {
+            encodedCQCounters_ = java.util.Collections.unmodifiableList(encodedCQCounters_);
+            bitField1_ = (bitField1_ & ~0x00000002);
+          }
+          result.encodedCQCounters_ = encodedCQCounters_;
+        } else {
+          result.encodedCQCounters_ = encodedCQCountersBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5441,6 +5695,35 @@ public final class PTableProtos {
         if (other.hasParentNameBytes()) {
           setParentNameBytes(other.getParentNameBytes());
         }
+        if (other.hasStorageScheme()) {
+          setStorageScheme(other.getStorageScheme());
+        }
+        if (encodedCQCountersBuilder_ == null) {
+          if (!other.encodedCQCounters_.isEmpty()) {
+            if (encodedCQCounters_.isEmpty()) {
+              encodedCQCounters_ = other.encodedCQCounters_;
+              bitField1_ = (bitField1_ & ~0x00000002);
+            } else {
+              ensureEncodedCQCountersIsMutable();
+              encodedCQCounters_.addAll(other.encodedCQCounters_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.encodedCQCounters_.isEmpty()) {
+            if (encodedCQCountersBuilder_.isEmpty()) {
+              encodedCQCountersBuilder_.dispose();
+              encodedCQCountersBuilder_ = null;
+              encodedCQCounters_ = other.encodedCQCounters_;
+              bitField1_ = (bitField1_ & ~0x00000002);
+              encodedCQCountersBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getEncodedCQCountersFieldBuilder() : null;
+            } else {
+              encodedCQCountersBuilder_.addAllMessages(other.encodedCQCounters_);
+            }
+          }
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5494,6 +5777,12 @@ public final class PTableProtos {
             return false;
           }
         }
+        for (int i = 0; i < getEncodedCQCountersCount(); i++) {
+          if (!getEncodedCQCounters(i).isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -5515,6 +5804,7 @@ public final class PTableProtos {
         return this;
       }
       private int bitField0_;
+      private int bitField1_;
 
       // required bytes schemaNameBytes = 1;
       private com.google.protobuf.ByteString schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -7168,101 +7458,1006 @@ public final class PTableProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:PTable)
-    }
-
-    static {
-      defaultInstance = new PTable(true);
-      defaultInstance.initFields();
-    }
-
-    // @@protoc_insertion_point(class_scope:PTable)
+      // optional bytes storageScheme = 34;
+      private com.google.protobuf.ByteString storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes storageScheme = 34;</code>
+       */
+      public boolean hasStorageScheme() {
+        return ((bitField1_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional bytes storageScheme = 34;</code>
+       */
+      public com.google.protobuf.ByteString getStorageScheme() {
+        return storageScheme_;
+      }
+      /**
+       * <code>optional bytes storageScheme = 34;</code>
+       */
+      public Builder setStorageScheme(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
   }
+  bitField1_ |= 0x00000001;
+        storageScheme_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes storageScheme = 34;</code>
+       */
+      public Builder clearStorageScheme() {
+        bitField1_ = (bitField1_ & ~0x00000001);
+        storageScheme_ = getDefaultInstance().getStorageScheme();
+        onChanged();
+        return this;
+      }
 
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_PColumn_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_PColumn_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_PTableStats_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_PTableStats_fieldAccessorTable;
-  private static com.google.protobuf.Descriptors.Descriptor
-    internal_static_PTable_descriptor;
-  private static
-    com.google.protobuf.GeneratedMessage.FieldAccessorTable
-      internal_static_PTable_fieldAccessorTable;
+      // repeated .EncodedCQCounter encodedCQCounters = 35;
+      private java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> encodedCQCounters_ =
+        java.util.Collections.emptyList();
+      private void ensureEncodedCQCountersIsMutable() {
+        if (!((bitField1_ & 0x00000002) == 0x00000002)) {
+          encodedCQCounters_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter>(encodedCQCounters_);
+          bitField1_ |= 0x00000002;
+         }
+      }
 
-  public static com.google.protobuf.Descriptors.FileDescriptor
-      getDescriptor() {
-    return descriptor;
-  }
-  private static com.google.protobuf.Descriptors.FileDescriptor
-      descriptor;
-  static {
-    java.lang.String[] descriptorData = {
-      "\n\014PTable.proto\032\021PGuidePosts.proto\"\246\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" +
-      "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \002" +
-      "(\005\022\021\n\tarraySize\030\t \001(\005\022\024\n\014viewConstant\030\n " +
-      "\001(\014\022\026\n\016viewReferenced\030\013 \001(\010\022\022\n\nexpressio" +
-      "n\030\014 \001(\t\022\026\n\016isRowTimestamp\030\r \001(\010\022\021\n\tisDyn" +
-      "amic\030\016 \001(\010\"\232\001\n\013PTableStats\022\013\n\003key\030\001 \002(\014\022" +
-      "\016\n\006values\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003",
-      " \001(\003\022\025\n\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePost" +
-      "sCount\030\005 \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGui" +
-      "dePosts\"\217\006\n\006PTable\022\027\n\017schemaNameBytes\030\001 " +
-      "\002(\014\022\026\n\016tableNameBytes\030\002 \002(\014\022\036\n\ttableType" +
-      "\030\003 \002(\0162\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022" +
-      "\026\n\016sequenceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002" +
-      "(\003\022\023\n\013pkNameBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002" +
-      "(\005\022\031\n\007columns\030\t \003(\0132\010.PColumn\022\030\n\007indexes" +
-      "\030\n \003(\0132\007.PTable\022\027\n\017isImmutableRows\030\013 \002(\010" +
-      "\022\032\n\022dataTableNameBytes\030\r \001(\014\022\031\n\021defaultF",
-      "amilyName\030\016 \001(\014\022\022\n\ndisableWAL\030\017 \002(\010\022\023\n\013m" +
-      "ultiTenant\030\020 \002(\010\022\020\n\010viewType\030\021 \001(\014\022\025\n\rvi" +
-      "ewStatement\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\nstoreNulls\030\030 \001(\010\022\027\n\017baseColumnCoun" +
-      "t\030\031 \001(\005\022\036\n\026rowKeyOrderOptimizable\030\032 \001(\010\022" +
-      "\025\n\rtransactional\030\033 \001(\010\022\034\n\024updateCacheFre" +
-      "quency\030\034 \001(\003\022\035\n\025indexDisableTimestamp\030\035 " +
-      "\001(\003\022\031\n\021isNamespaceMapped\030\036 \001(\010\022\034\n\024autoPa",
-      "rititonSeqName\030\037 \001(\t\022\032\n\022isAppendOnlySche" +
-      "ma\030  \001(\010\022\027\n\017parentNameBytes\030! \001(\014*A\n\nPTa" +
-      "bleType\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.phoen" +
-      "ix.coprocessor.generatedB\014PTableProtosH\001" +
-      "\210\001\001\240\001\001"
-    };
-    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
-      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
-        public com.google.protobuf.ExtensionRegistry assignDescriptors(
-            com.google.protobuf.Descriptors.FileDescriptor root) {
-          descriptor = root;
-          internal_static_PColumn_descriptor =
-            getDescriptor().getMessageTypes().get(0);
-          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", });
-          internal_static_PTableStats_descriptor =
-            getDescriptor().getMessageTypes().get(1);
-          internal_static_PTableStats_fieldAccessorTable = new
-            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
-              internal_static_PTableStats_descriptor,
-              new java.lang.String[] { "Key", "Values", "GuidePostsByteCount", "KeyBytesCount", "GuidePostsCount", "PGuidePosts", });
-          internal_static_PTable_descriptor =
-            getDescriptor().getMessageTypes().get(2);
-          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", });
+      private 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> encodedCQCountersBuilder_;
+
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> getEncodedCQCountersList() {
+        if (encodedCQCountersBuilder_ == null) {
+          return java.util.Collections.unmodifiableList(encodedCQCounters_);
+        } else {
+          return encodedCQCountersBuilder_.getMessageList();
+        }
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public int getEncodedCQCountersCount() {
+        if (encodedCQCountersBuilder_ == null) {
+          return encodedCQCounters_.size();
+        } else {
+          return encodedCQCountersBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index) {
+        if (encodedCQCountersBuilder_ == null) {
+          return encodedCQCounters_.get(index);
+        } else {
+          return encodedCQCountersBuilder_.getMessage(index);
+        }
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder setEncodedCQCounters(
+          int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter value) {
+        if (encodedCQCountersBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.set(index, value);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.setMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder setEncodedCQCounters(
+          int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder builderForValue) {
+        if (encodedCQCountersBuilder_ == null) {
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.set(index, builderForValue.build());
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.setMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder addEncodedCQCounters(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter value) {
+        if (encodedCQCountersBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.add(value);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.addMessage(value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder addEncodedCQCounters(
+          int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter value) {
+        if (encodedCQCountersBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.add(index, value);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.addMessage(index, value);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder addEncodedCQCounters(
+          org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder builderForValue) {
+        if (encodedCQCountersBuilder_ == null) {
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.add(builderForValue.build());
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.addMessage(builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder addEncodedCQCounters(
+          int index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder builderForValue) {
+        if (encodedCQCountersBuilder_ == null) {
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.add(index, builderForValue.build());
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.addMessage(index, builderForValue.build());
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder addAllEncodedCQCounters(
+          java.lang.Iterable<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> values) {
+        if (encodedCQCountersBuilder_ == null) {
+          ensureEncodedCQCountersIsMutable();
+          super.addAll(values, encodedCQCounters_);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.addAllMessages(values);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder clearEncodedCQCounters() {
+        if (encodedCQCountersBuilder_ == null) {
+          encodedCQCounters_ = java.util.Collections.emptyList();
+          bitField1_ = (bitField1_ & ~0x00000002);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public Builder removeEncodedCQCounters(int index) {
+        if (encodedCQCountersBuilder_ == null) {
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.remove(index);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder getEncodedCQCountersBuilder(
+          int index) {
+        return getEncodedCQCountersFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
+          int index) {
+        if (encodedCQCountersBuilder_ == null) {
+          return encodedCQCounters_.get(index);  } else {
+          return encodedCQCountersBuilder_.getMessageOrBuilder(index);
+        }
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
+           getEncodedCQCountersOrBuilderList() {
+        if (encodedCQCountersBuilder_ != null) {
+          return encodedCQCountersBuilder_.getMessageOrBuilderList();
+        } else {
+          return java.util.Collections.unmodifiableList(encodedCQCounters_);
+        }
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</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>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder addEncodedCQCountersBuilder(
+          int index) {
+        return getEncodedCQCountersFieldBuilder().addBuilder(
+            index, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 35;</code>
+       */
+      public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder> 
+           getEncodedCQCountersBuilderList() {
+        return getEncodedCQCountersFieldBuilder().getBuilderList();
+      }
+      private 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> 
+          getEncodedCQCountersFieldBuilder() {
+        if (encodedCQCountersBuilder_ == null) {
+          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),
+                  getParentForChildren(),
+                  isClean());
+          encodedCQCounters_ = null;
+        }
+        return encodedCQCountersBuilder_;
+      }
+
+      // @@protoc_insertion_point(builder_scope:PTable)
+    }
+
+    static {
+      defaultInstance = new PTable(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:PTable)
+  }
+
+  public interface EncodedCQCounterOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required string colFamily = 1;
+    /**
+     * <code>required string colFamily = 1;</code>
+     */
+    boolean hasColFamily();
+    /**
+     * <code>required string colFamily = 1;</code>
+     */
+    java.lang.String getColFamily();
+    /**
+     * <code>required string colFamily = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getColFamilyBytes();
+
+    // required int32 counter = 2;
+    /**
+     * <code>required int32 counter = 2;</code>
+     */
+    boolean hasCounter();
+    /**
+     * <code>required int32 counter = 2;</code>
+     */
+    int getCounter();
+  }
+  /**
+   * Protobuf type {@code EncodedCQCounter}
+   */
+  public static final class EncodedCQCounter extends
+      com.google.protobuf.GeneratedMessage
+      implements EncodedCQCounterOrBuilder {
+    // Use EncodedCQCounter.newBuilder() to construct.
+    private EncodedCQCounter(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private EncodedCQCounter(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final EncodedCQCounter defaultInstance;
+    public static EncodedCQCounter getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public EncodedCQCounter getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private EncodedCQCounter(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              colFamily_ = input.readBytes();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              counter_ = input.readInt32();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.PTableProtos.internal_static_EncodedCQCounter_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.PTableProtos.internal_static_EncodedCQCounter_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.class, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<EncodedCQCounter> PARSER =
+        new com.google.protobuf.AbstractParser<EncodedCQCounter>() {
+      public EncodedCQCounter parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new EncodedCQCounter(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<EncodedCQCounter> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required string colFamily = 1;
+    public static final int COLFAMILY_FIELD_NUMBER = 1;
+    private java.lang.Object colFamily_;
+    /**
+     * <code>required string colFamily = 1;</code>
+     */
+    public boolean hasColFamily() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required string colFamily = 1;</code>
+     */
+    public java.lang.String getColFamily() {
+      java.lang.Object ref = colFamily_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          colFamily_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string colFamily = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getColFamilyBytes() {
+      java.lang.Object ref = colFamily_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        colFamily_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required int32 counter = 2;
+    public static final int COUNTER_FIELD_NUMBER = 2;
+    private int counter_;
+    /**
+     * <code>required int32 counter = 2;</code>
+     */
+    public boolean hasCounter() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required int32 counter = 2;</code>
+     */
+    public int getCounter() {
+      return counter_;
+    }
+
+    private void initFields() {
+      colFamily_ = "";
+      counter_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasColFamily()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasCounter()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getColFamilyBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeInt32(2, counter_);
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getColFamilyBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeInt32Size(2, counter_);
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter other = (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter) obj;
+
+      boolean result = true;
+      result = result && (hasColFamily() == other.hasColFamily());
+      if (hasColFamily()) {
+        result = result && getColFamily()
+            .equals(other.getColFamily());
+      }
+      result = result && (hasCounter() == other.hasCounter());
+      if (hasCounter()) {
+        result = result && (getCounter()
+            == other.getCounter());
+      }
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (hasColFamily()) {
+        hash = (37 * hash) + COLFAMILY_FIELD_NUMBER;
+        hash = (53 * hash) + getColFamily().hashCode();
+      }
+      if (hasCounter()) {
+        hash = (37 * hash) + COUNTER_FIELD_NUMBER;
+        hash = (53 * hash) + getCounter();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code EncodedCQCounter}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.PTableProtos.internal_static_EncodedCQCounter_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.PTableProtos.internal_static_EncodedCQCounter_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.class, org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        colFamily_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        counter_ = 0;
+        bitField0_ = (bitField0_ & ~0x00000002);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.PTableProtos.internal_static_EncodedCQCounter_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter build() {
+        org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter buildPartial() {
+        org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter result = new org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.colFamily_ = colFamily_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.counter_ = counter_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter other) {
+        if (other == org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.getDefaultInstance()) return this;
+        if (other.hasColFamily()) {
+          bitField0_ |= 0x00000001;
+          colFamily_ = other.colFamily_;
+          onChanged();
+        }
+        if (other.hasCounter()) {
+          setCounter(other.getCounter());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasColFamily()) {
+          
+          return false;
+        }
+        if (!hasCounter()) {
+          
+          return false;
+        }
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required string colFamily = 1;
+      private java.lang.Object colFamily_ = "";
+      /**
+       * <code>required string colFamily = 1;</code>
+       */
+      public boolean hasColFamily() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required string colFamily = 1;</code>
+       */
+      public java.lang.String getColFamily() {
+        java.lang.Object ref = colFamily_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          colFamily_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string colFamily = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getColFamilyBytes() {
+        java.lang.Object ref = colFamily_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          colFamily_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string colFamily = 1;</code>
+       */
+      public Builder setColFamily(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        colFamily_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string colFamily = 1;</code>
+       */
+      public Builder clearColFamily() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        colFamily_ = getDefaultInstance().getColFamily();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string colFamily = 1;</code>
+       */
+      public Builder setColFamilyBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        colFamily_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required int32 counter = 2;
+      private int counter_ ;
+      /**
+       * <code>required int32 counter = 2;</code>
+       */
+      public boolean hasCounter() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required int32 counter = 2;</code>
+       */
+      public int getCounter() {
+        return counter_;
+      }
+      /**
+       * <code>required int32 counter = 2;</code>
+       */
+      public Builder setCounter(int value) {
+        bitField0_ |= 0x00000002;
+        counter_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required int32 counter = 2;</code>
+       */
+      public Builder clearCounter() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        counter_ = 0;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:EncodedCQCounter)
+    }
+
+    static {
+      defaultInstance = new EncodedCQCounter(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:EncodedCQCounter)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_PColumn_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_PColumn_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_PTableStats_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_PTableStats_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_PTable_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_PTable_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_EncodedCQCounter_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_EncodedCQCounter_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\014PTable.proto\032\021PGuidePosts.proto\"\277\002\n\007PC" +
+      "olumn\022\027\n\017columnNameBytes\030\001 \002(\014\022\027\n\017family" +
+      "NameBytes\030\002 \001(\014\022\020\n\010dataType\030\003 \002(\t\022\021\n\tmax" +
+      "Length\030\004 \001(\005\022\r\n\005scale\030\005 \001(\005\022\020\n\010nullable\030" +
+      "\006 \002(\010\022\020\n\010position\030\007 \002(\005\022\021\n\tsortOrder\030\010 \002" +
+      "(\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"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_PColumn_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          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", });
+          internal_static_PTableStats_descriptor =
+            getDescriptor().getMessageTypes().get(1);
+          internal_static_PTableStats_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_PTableStats_descriptor,
+              new java.lang.String[] { "Key", "Values", "GuidePostsByteCount", "KeyBytesCount", "GuidePostsCount", "PGuidePosts", });
+          internal_static_PTable_descriptor =
+            getDescriptor().getMessageTypes().get(2);
+          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", });
+          internal_static_EncodedCQCounter_descriptor =
+            getDescriptor().getMessageTypes().get(3);
+          internal_static_EncodedCQCounter_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_EncodedCQCounter_descriptor,
+              new java.lang.String[] { "ColFamily", "Counter", });
           return null;
         }
       };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
index 352b533..fb5af32 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/BaseQueryPlan.java
@@ -64,11 +64,13 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.trace.TracingIterator;
 import org.apache.phoenix.trace.util.Tracing;
 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.SQLCloseable;
@@ -307,10 +309,6 @@ public abstract class BaseQueryPlan implements QueryPlan {
             // project is not present in the index then we need to skip this plan.
             if (!dataColumns.isEmpty()) {
                 // Set data columns to be join back from data table.
-                serializeDataTableColumnsToJoin(scan, dataColumns);
-                KeyValueSchema schema = ProjectedColumnExpression.buildSchema(dataColumns);
-                // Set key value schema of the data columns.
-                serializeSchemaIntoScan(scan, schema);
                 PTable parentTable = context.getCurrentTable().getTable();
                 String parentSchemaName = parentTable.getParentSchemaName().getString();
                 String parentTableName = parentTable.getParentTableName().getString();
@@ -321,6 +319,12 @@ public abstract class BaseQueryPlan implements QueryPlan {
                             FACTORY.namedTable(null, TableName.create(parentSchemaName, parentTableName)),
                             context.getConnection()).resolveTable(parentSchemaName, parentTableName);
                 PTable dataTable = dataTableRef.getTable();
+                // Set data columns to be join back from data table.
+                serializeDataTableColumnsToJoin(scan, dataColumns, dataTable);
+                KeyValueSchema schema = ProjectedColumnExpression.buildSchema(dataColumns);
+                // Set key value schema of the data columns.
+                serializeSchemaIntoScan(scan, schema);
+                
                 // Set index maintainer of the local index.
                 serializeIndexMaintainerIntoScan(scan, dataTable);
                 // Set view constants if exists.
@@ -423,14 +427,21 @@ public abstract class BaseQueryPlan implements QueryPlan {
         }
     }
 
-    private void serializeDataTableColumnsToJoin(Scan scan, Set<PColumn> dataColumns) {
+    private void serializeDataTableColumnsToJoin(Scan scan, Set<PColumn> dataColumns, PTable dataTable) {
         ByteArrayOutputStream stream = new ByteArrayOutputStream();
         try {
             DataOutputStream output = new DataOutputStream(stream);
+            boolean storeColsInSingleCell = dataTable.getStorageScheme() == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL;
+            if (storeColsInSingleCell) {
+                // if storeColsInSingleCell is true all columns of a given column family are stored in a single cell
+                scan.setAttribute(BaseScannerRegionObserver.COLUMNS_STORED_IN_SINGLE_CELL, QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+            }
             WritableUtils.writeVInt(output, dataColumns.size());
             for (PColumn column : dataColumns) {
-                Bytes.writeByteArray(output, column.getFamilyName().getBytes());
-                Bytes.writeByteArray(output, column.getName().getBytes());
+                byte[] cf = column.getFamilyName().getBytes();
+                byte[] cq = EncodedColumnsUtil.getColumnQualifier(column, dataTable);
+                Bytes.writeByteArray(output, cf);
+                Bytes.writeByteArray(output, cq);
             }
             scan.setAttribute(BaseScannerRegionObserver.DATA_TABLE_COLUMNS_TO_JOIN, stream.toByteArray());
         } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
index cb66968..a030150 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/MutationState.java
@@ -585,7 +585,7 @@ public class MutationState implements SQLCloseable {
                 List<Mutation> indexMutations;
                 try {
                     indexMutations =
-                    		IndexUtil.generateIndexData(table, index, mutationsPertainingToIndex,
+                    		IndexUtil.generateIndexData(table, index, values, mutationsPertainingToIndex,
                                 connection.getKeyValueBuilder(), connection);
                     // we may also have to include delete mutations for immutable tables if we are not processing all the tables in the mutations map
                     if (!sendAll) {
@@ -619,6 +619,7 @@ public class MutationState implements SQLCloseable {
         Iterator<Map.Entry<ImmutableBytesPtr, RowMutationState>> iterator =
                 values.entrySet().iterator();
         long timestampToUse = timestamp;
+        Map<ImmutableBytesPtr, RowMutationState> modifiedValues = Maps.newHashMap();
         while (iterator.hasNext()) {
             Map.Entry<ImmutableBytesPtr, RowMutationState> rowEntry = iterator.next();
             byte[] onDupKeyBytes = rowEntry.getValue().getOnDupKeyBytes();
@@ -628,6 +629,10 @@ public class MutationState implements SQLCloseable {
             if (tableWithRowTimestampCol) {
                 RowTimestampColInfo rowTsColInfo = state.getRowTimestampColInfo();
                 if (rowTsColInfo.useServerTimestamp()) {
+                	// since we are about to modify the byte[] stored in key (which changes its hashcode)
+                	// we need to remove the entry from the values map and add a new entry with the modified byte[]
+                	modifiedValues.put(key, state);
+                	iterator.remove();
                     // regenerate the key with this timestamp.
                     key = getNewRowKeyWithRowTimestamp(key, timestampToUse, table);
                 } else {
@@ -668,6 +673,7 @@ public class MutationState implements SQLCloseable {
             if (mutationsPertainingToIndex != null) mutationsPertainingToIndex
                     .addAll(rowMutationsPertainingToIndex);
         }
+        values.putAll(modifiedValues);
     }
     
     /**
@@ -806,7 +812,7 @@ public class MutationState implements SQLCloseable {
                 }
                 for (PColumn column : columns) {
                     if (column != null) {
-                        resolvedTable.getColumnFamily(column.getFamilyName().getString()).getColumn(column.getName().getString());
+                        resolvedTable.getColumnFamily(column.getFamilyName().getString()).getPColumnForColumnName(column.getName().getString());
                     }
                 }
             }
@@ -1480,8 +1486,8 @@ public class MutationState implements SQLCloseable {
         byte[] getOnDupKeyBytes() {
             return onDupKeyBytes;
         }
-        
-        Map<PColumn, byte[]> getColumnValues() {
+
+        public Map<PColumn, byte[]> getColumnValues() {
             return columnValues;
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
index 592b68e..127af96 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
@@ -17,6 +17,9 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_QUALIFIER;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -51,9 +54,6 @@ import org.apache.phoenix.util.SchemaUtil;
 import com.google.common.base.Preconditions;
 
 public class TupleProjector {    
-    public static final byte[] VALUE_COLUMN_FAMILY = Bytes.toBytes("_v");
-    public static final byte[] VALUE_COLUMN_QUALIFIER = new byte[0];
-    
     private static final String SCAN_PROJECTOR = "scanProjector";
     
     private final KeyValueSchema schema;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayColumnExpression.java
new file mode 100644
index 0000000..f4616da
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayColumnExpression.java
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.expression;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.compile.CreateTableCompiler.ViewWhereExpressionVisitor;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.SchemaUtil;
+
+/**
+ * 
+ * Class to access a column that is stored in a KeyValue that contains all
+ * columns for a given column family (stored in an array).
+ *
+ */
+public class ArrayColumnExpression extends KeyValueColumnExpression {
+    
+    private int encodedCQ;
+    private String displayName;
+    
+    public ArrayColumnExpression() {
+    }
+    
+    public ArrayColumnExpression(PDatum column, byte[] cf, int encodedCQ) {
+        super(column, cf, cf);
+        this.encodedCQ = encodedCQ;
+    }
+    
+    public ArrayColumnExpression(PColumn column, String displayName, boolean encodedColumnName) {
+        super(column, column.getFamilyName().getBytes(), column.getFamilyName().getBytes());
+        this.displayName = SchemaUtil.getColumnDisplayName(column.getFamilyName().getString(), column.getName().getString());
+        this.encodedCQ = column.getEncodedColumnQualifier();
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+    	if (!super.evaluate(tuple, ptr)) {
+            return false;
+        } else if (ptr.getLength() == 0) { 
+        	return true; 
+        }
+
+        // Given a ptr to the entire array, set ptr to point to a particular element within that array
+        // given the type of an array element (see comments in PDataTypeForArray)
+    	PArrayDataType.positionAtArrayElement(ptr, encodedCQ, PVarbinary.INSTANCE, null);
+        return true;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        encodedCQ = WritableUtils.readVInt(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, encodedCQ);
+    }
+    
+    public KeyValueColumnExpression getKeyValueExpression() {
+    	final boolean isNullable = isNullable();
+    	final SortOrder sortOrder = getSortOrder();
+    	final Integer scale = getScale();
+    	final Integer maxLength = getMaxLength();
+    	final PDataType datatype = getDataType();
+        return new KeyValueColumnExpression(new PDatum() {
+			
+			@Override
+			public boolean isNullable() {
+				return isNullable;
+			}
+			
+			@Override
+			public SortOrder getSortOrder() {
+				return sortOrder;
+			}
+			
+			@Override
+			public Integer getScale() {
+				return scale;
+			}
+			
+			@Override
+			public Integer getMaxLength() {
+				return maxLength;
+			}
+			
+			@Override
+			public PDataType getDataType() {
+				return datatype;
+			}
+		}, getColumnFamily(), getEncodedColumnQualifier());
+    }
+    
+    @Override
+    public String toString() {
+        return displayName;
+    }
+    
+    public byte[] getEncodedColumnQualifier() {
+        return EncodedColumnsUtil.getEncodedColumnQualifier(encodedCQ);
+    }
+    
+    @Override
+    public <T> T accept(ExpressionVisitor<T> visitor) {
+        //FIXME: this is ugly but can't think of a good solution.
+        if (visitor instanceof ViewWhereExpressionVisitor) {
+            return visitor.visit(this);
+        } else {
+            return super.accept(visitor);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
index c2f4dd2..783e962 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
@@ -117,7 +117,7 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
                             offsetPos[i] = byteStream.size();
                             oStream.write(ptr.get(), ptr.getOffset(), ptr.getLength());
                             oStream.write(PArrayDataType.getSeparatorByte(rowKeyOrderOptimizable, getSortOrder()));
-                            nNulls=0;
+                            nNulls = 0;
                         }
                     } else { // No nulls for fixed length
                         oStream.write(ptr.get(), ptr.getOffset(), ptr.getLength());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
index 658605e..006777b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ExpressionType.java
@@ -178,9 +178,8 @@ public enum ExpressionType {
     CeilYearExpression(CeilYearExpression.class),
     DayOfWeekFunction(DayOfWeekFunction.class),
     DayOfYearFunction(DayOfYearFunction.class),
-    DefaultValueExpression(DefaultValueExpression.class);
-
-
+    DefaultValueExpression(DefaultValueExpression.class),
+    ArrayColumnExpression(ArrayColumnExpression.class);
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
index 4b5fdbb..6170418 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/KeyValueColumnExpression.java
@@ -28,6 +28,7 @@ import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
 
@@ -41,13 +42,13 @@ import org.apache.phoenix.util.SchemaUtil;
 public class KeyValueColumnExpression extends ColumnExpression {
     private byte[] cf;
     private byte[] cq;
-    private String displayName; // client-side only
+    private String displayName; // client-side only. TODO: samarth see what can you do for encoded column names.
 
     public KeyValueColumnExpression() {
     }
 
-    public KeyValueColumnExpression(PColumn column) {
-        this(column, null);
+    public KeyValueColumnExpression(PColumn column, boolean encodedColumnName) {
+        this(column, null, encodedColumnName);
     }
 
     public KeyValueColumnExpression(PDatum column, byte[] cf, byte[] cq) {
@@ -56,18 +57,18 @@ public class KeyValueColumnExpression extends ColumnExpression {
         this.cq = cq;
     }
 
-    public KeyValueColumnExpression(PColumn column, String displayName) {
+    public KeyValueColumnExpression(PColumn column, String displayName, boolean encodedColumnName) {
         super(column);
         this.cf = column.getFamilyName().getBytes();
-        this.cq = column.getName().getBytes();
+        this.cq = EncodedColumnsUtil.getColumnQualifier(column, encodedColumnName);
         this.displayName = displayName;
     }
 
     public byte[] getColumnFamily() {
         return cf;
     }
-
-    public byte[] getColumnName() {
+    
+    public byte[] getColumnQualifier() {
         return cq;
     }
 
@@ -120,7 +121,7 @@ public class KeyValueColumnExpression extends ColumnExpression {
     }
 
     @Override
-    public final <T> T accept(ExpressionVisitor<T> visitor) {
+    public <T> T accept(ExpressionVisitor<T> visitor) {
         return visitor.visit(this);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
index 90882a2..f20d7e2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/LiteralExpression.java
@@ -23,6 +23,7 @@ import java.io.IOException;
 import java.sql.SQLException;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.schema.IllegalDataException;
@@ -214,6 +215,11 @@ public class LiteralExpression extends BaseTerminalExpression {
 
     public LiteralExpression() {
     }
+    
+    public LiteralExpression(byte[] byteValue) {
+        this.byteValue = byteValue!=null ? byteValue : ByteUtil.EMPTY_BYTE_ARRAY;
+        this.determinism = Determinism.ALWAYS;
+    }
 
     private LiteralExpression(PDataType type, Determinism determinism) {
         this(null, type, ByteUtil.EMPTY_BYTE_ARRAY, determinism);
@@ -242,7 +248,10 @@ public class LiteralExpression extends BaseTerminalExpression {
     
     @Override
     public String toString() {
-        if (value == null) {
+        if (value == null && byteValue!=null) {
+            return Bytes.toStringBinary(byteValue);
+        }
+        else if (value == null) {
             return "null";
         }
         // TODO: move into PDataType?

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


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 064137e..515e428 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
@@ -223,7 +223,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();
@@ -238,7 +238,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(), oldColumn.getExpressionStr(), 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(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getEncodedColumnQualifier());
                 columns.add(newColumn);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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/56c17679/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 01e8afe..d3b11b2 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
@@ -17,7 +17,15 @@
  */
 package org.apache.phoenix.schema;
 
+import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.annotation.Nullable;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -129,7 +137,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;
 
@@ -153,6 +161,35 @@ public interface PTable extends PMetaDataEntity {
             return LinkType.values()[serializedValue-1];
         }
     }
+    
+    public enum StorageScheme {
+        ENCODED_COLUMN_NAMES((byte)1),
+        NON_ENCODED_COLUMN_NAMES((byte)2),
+        COLUMNS_STORED_IN_SINGLE_CELL((byte)3);
+
+        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();
@@ -208,7 +245,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[] cf, byte[] cq) throws ColumnNotFoundException, AmbiguousColumnException; 
     
     /**
      * Get the PK column with the given name.
@@ -345,7 +391,6 @@ public interface PTable extends PMetaDataEntity {
      */
     int getRowTimestampColPos();
     long getUpdateCacheFrequency();
-
     boolean isNamespaceMapped();
     
     /**
@@ -359,4 +404,92 @@ public interface PTable extends PMetaDataEntity {
      * you are also not allowed to delete the table  
      */
     boolean isAppendOnlySchema();
+    StorageScheme getStorageScheme();
+    EncodedCQCounter getEncodedCQCounter();
+    
+    /**
+     * Class to help track encoded column qualifier counters per column family.
+     */
+    public class EncodedCQCounter {
+        
+        private final Map<String, Integer> familyCounters = new HashMap<>();
+        
+        /**
+         * Copy constructor
+         * @param counterToCopy
+         * @return copy of the passed counter
+         */
+        public static EncodedCQCounter copy(EncodedCQCounter counterToCopy) {
+            EncodedCQCounter cqCounter = new EncodedCQCounter();
+            for (Entry<String, Integer> e : counterToCopy.values().entrySet()) {
+                cqCounter.setValue(e.getKey(), e.getValue());
+            }
+            return cqCounter;
+        }
+        
+        public static final EncodedCQCounter NULL_COUNTER = new EncodedCQCounter() {
+
+            @Override
+            public Integer getNextQualifier(String columnFamily) {
+                return null;
+            }
+
+            @Override
+            public void setValue(String columnFamily, Integer value) {
+            }
+
+            @Override
+            public boolean increment(String columnFamily) {
+                return false;
+            }
+
+            @Override
+            public Map<String, Integer> values() {
+                return Collections.emptyMap();
+            }
+
+        };
+        
+        /**
+         * Get the next qualifier to be used for the column family.
+         * This method also ends up initializing the counter if the
+         * column family already doesn't have one.
+         */
+        @Nullable
+        public Integer getNextQualifier(String columnFamily) {
+            Integer counter = familyCounters.get(columnFamily);
+            if (counter == null) {
+                counter = ENCODED_CQ_COUNTER_INITIAL_VALUE;
+                familyCounters.put(columnFamily, counter);
+            }
+            return counter;
+        }
+        
+        public void setValue(String columnFamily, Integer value) {
+            familyCounters.put(columnFamily, value);
+        }
+        
+        /**
+         * 
+         * @param columnFamily
+         * @return true if the counter was incrememnted, false otherwise.
+         */
+        public boolean increment(String columnFamily) {
+            if (columnFamily == null) {
+                return false;
+            }
+            Integer counter = familyCounters.get(columnFamily);
+            if (counter == null) {
+                counter = ENCODED_CQ_COUNTER_INITIAL_VALUE;
+            }
+            counter++;
+            familyCounters.put(columnFamily, counter);
+            return true;
+        }
+        
+        public Map<String, Integer> values()  {
+            return Collections.unmodifiableMap(familyCounters);
+        }
+        
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 98a0b99..1134e06 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.schema;
 import static org.apache.phoenix.hbase.index.util.KeyValueBuilder.addQuietly;
 import static org.apache.phoenix.hbase.index.util.KeyValueBuilder.deleteQuietly;
 import static org.apache.phoenix.schema.SaltingUtil.SALTING_COLUMN;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
 
 import java.io.IOException;
 import java.sql.DriverManager;
@@ -30,6 +31,7 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
@@ -41,11 +43,14 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.ExpressionCompiler;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
+import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
@@ -56,13 +61,16 @@ import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
+import org.apache.phoenix.schema.tuple.BaseTuple;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
+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.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
@@ -79,6 +87,7 @@ import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+
 /**
  *
  * Base class for PTable implementors.  Provides abstraction for
@@ -106,7 +115,8 @@ public class PTableImpl implements PTable {
     private List<PColumnFamily> families;
     private Map<byte[], PColumnFamily> familyByBytes;
     private Map<String, PColumnFamily> familyByString;
-    private ListMultimap<String,PColumn> columnsByName;
+    private ListMultimap<String, PColumn> columnsByName;
+    private ListMultimap<Pair<String, Integer>, PColumn> kvColumnsByEncodedColumnNames;
     private PName pkName;
     private Integer bucketNum;
     private RowKeySchema rowKeySchema;
@@ -138,6 +148,8 @@ public class PTableImpl implements PTable {
     private boolean isNamespaceMapped;
     private String autoPartitionSeqName;
     private boolean isAppendOnlySchema;
+    private StorageScheme storageScheme;
+    private EncodedCQCounter encodedCQCounter;
 
     public PTableImpl() {
         this.indexes = Collections.emptyList();
@@ -169,8 +181,9 @@ public class PTableImpl implements PTable {
         this.isNamespaceMapped = isNamespaceMapped;
     }
     
+    // For indexes stored in shared physical tables
     public PTableImpl(PName tenantId, PName schemaName, PName tableName, long timestamp, List<PColumnFamily> families, 
-            List<PColumn> columns, List<PName> physicalNames, Short viewIndexId, boolean multiTenant, boolean isNamespaceMpped) throws SQLException { // For indexes stored in shared physical tables
+            List<PColumn> columns, List<PName> physicalNames, Short viewIndexId, boolean multiTenant, boolean isNamespaceMpped, StorageScheme storageScheme, EncodedCQCounter encodedCQCounter) throws SQLException {
         this.pkColumns = this.allColumns = Collections.emptyList();
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
         this.indexes = Collections.emptyList();
@@ -184,7 +197,7 @@ public class PTableImpl implements PTable {
         init(tenantId, this.schemaName, this.tableName, PTableType.INDEX, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
             this.schemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
             null, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-            isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMpped, null, false);
+            isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMpped, null, false, storageScheme, encodedCQCounter);
     }
 
     public PTableImpl(long timeStamp) { // For delete marker
@@ -228,7 +241,7 @@ public class PTableImpl implements PTable {
                     indexes, table.isImmutableRows(), physicalNames, table.getDefaultFamilyName(), viewStatement,
                     table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                     table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), updateCacheFrequency,
-                    table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                    table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
         }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes, PName parentSchemaName, String viewStatement) throws SQLException {
@@ -238,7 +251,7 @@ public class PTableImpl implements PTable {
                 indexes, table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), viewStatement,
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, Collection<PColumn> columns) throws SQLException {
@@ -248,7 +261,7 @@ public class PTableImpl implements PTable {
                 table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns) throws SQLException {
@@ -258,7 +271,7 @@ public class PTableImpl implements PTable {
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(), table.isWALDisabled(),
                 table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
-                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows) throws SQLException {
@@ -268,7 +281,7 @@ public class PTableImpl implements PTable {
                 table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(),
                 table.getIndexType(), table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(),
-                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
     
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled,
@@ -279,7 +292,7 @@ public class PTableImpl implements PTable {
                 table.getIndexes(), isImmutableRows, table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 isWalDisabled, isMultitenant, storeNulls, table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), isTransactional, updateCacheFrequency, table.getIndexDisableTimestamp(), 
-                isNamespaceMapped, table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                isNamespaceMapped, table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
     
     public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
@@ -290,7 +303,7 @@ public class PTableImpl implements PTable {
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(),
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, boolean rowKeyOrderOptimizable) throws SQLException {
@@ -301,7 +314,7 @@ public class PTableImpl implements PTable {
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), rowKeyOrderOptimizable, table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), 
-                table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table) throws SQLException {
@@ -312,7 +325,7 @@ public class PTableImpl implements PTable {
                 table.isImmutableRows(), table.getPhysicalNames(), table.getDefaultFamilyName(), table.getViewStatement(),
                 table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(), table.getViewIndexId(), table.getIndexType(),
                 table.getBaseColumnCount(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), 
-                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -321,12 +334,12 @@ public class PTableImpl implements PTable {
             boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression,
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
-            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema) throws SQLException {
+            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, StorageScheme storageScheme, EncodedCQCounter encodedCQCounter) throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns, dataSchemaName,
                 dataTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
                 indexType, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, rowKeyOrderOptimizable, isTransactional,
-                updateCacheFrequency,indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema);
+                updateCacheFrequency,indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, encodedCQCounter);
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -336,13 +349,13 @@ public class PTableImpl implements PTable {
             boolean disableWAL, boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
             int baseColumnCount, long indexDisableTimestamp, boolean isNamespaceMapped,
-            String autoPartitionSeqName, boolean isAppendOnlySchema)
+            String autoPartitionSeqName, boolean isAppendOnlySchema, StorageScheme storageScheme, EncodedCQCounter encodedCQCounter)
             throws SQLException {
         return new PTableImpl(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName,
                 bucketNum, columns, dataSchemaName, dataTableName, indexes, isImmutableRows, physicalNames,
                 defaultFamilyName, viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId,
                 indexType, baseColumnCount, rowKeyOrderOptimizable, isTransactional, updateCacheFrequency, 
-                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema);
+                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, encodedCQCounter);
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
@@ -351,11 +364,11 @@ public class PTableImpl implements PTable {
             List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL, boolean multiTenant,
             boolean storeNulls, ViewType viewType, Short viewIndexId, IndexType indexType,
             int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency,
-            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema) throws SQLException {
+            long indexDisableTimestamp, boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, StorageScheme storageScheme, EncodedCQCounter encodedCQCounter) throws SQLException {
         init(tenantId, schemaName, tableName, type, state, timeStamp, sequenceNumber, pkName, bucketNum, columns,
                 parentSchemaName, parentTableName, indexes, isImmutableRows, physicalNames, defaultFamilyName,
                 viewExpression, disableWAL, multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-                isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema);
+                isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoPartitionSeqName, isAppendOnlySchema, storageScheme, encodedCQCounter);
     }
     
     @Override
@@ -389,7 +402,7 @@ public class PTableImpl implements PTable {
             List<PTable> indexes, boolean isImmutableRows, List<PName> physicalNames, PName defaultFamilyName, String viewExpression, boolean disableWAL,
             boolean multiTenant, boolean storeNulls, ViewType viewType, Short viewIndexId,
             IndexType indexType , int baseColumnCount, boolean rowKeyOrderOptimizable, boolean isTransactional, long updateCacheFrequency, long indexDisableTimestamp, 
-            boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema) throws SQLException {
+            boolean isNamespaceMapped, String autoPartitionSeqName, boolean isAppendOnlySchema, StorageScheme storageScheme, EncodedCQCounter encodedCQCounter) throws SQLException {
         Preconditions.checkNotNull(schemaName);
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         int estimatedSize = SizedUtil.OBJECT_SIZE * 2 + 23 * SizedUtil.POINTER_SIZE + 4 * SizedUtil.INT_SIZE + 2 * SizedUtil.LONG_SIZE + 2 * SizedUtil.INT_OBJECT_SIZE +
@@ -425,10 +438,12 @@ public class PTableImpl implements PTable {
         this.isNamespaceMapped = isNamespaceMapped;
         this.autoPartitionSeqName = autoPartitionSeqName;
         this.isAppendOnlySchema = isAppendOnlySchema;
+        this.storageScheme = storageScheme;
         List<PColumn> pkColumns;
         PColumn[] allColumns;
         
         this.columnsByName = ArrayListMultimap.create(columns.size(), 1);
+        this.kvColumnsByEncodedColumnNames = (EncodedColumnsUtil.usesEncodedColumnNames(storageScheme) ? ArrayListMultimap.<Pair<String, Integer>, PColumn>create(columns.size(), 1) : null);
         int numPKColumns = 0;
         if (bucketNum != null) {
             // Add salt column to allColumns and pkColumns, but don't add to
@@ -454,7 +469,26 @@ public class PTableImpl implements PTable {
                     if (Objects.equal(familyName, dupColumn.getFamilyName())) {
                         count++;
                         if (count > 1) {
-                            throw new ColumnAlreadyExistsException(null, name.getString(), columnName);
+                            throw new ColumnAlreadyExistsException(schemaName.getString(), name.getString(), columnName);
+                        }
+                    }
+                }
+            }
+            //TODO: samarth understand the implication of this.
+            if (kvColumnsByEncodedColumnNames != null) {
+                Integer cq = column.getEncodedColumnQualifier();
+                String cf = column.getFamilyName() != null ? column.getFamilyName().getString() : null;
+                if (cf != null && cq != null) {
+                    Pair<String, Integer> pair = new Pair<>(cf, cq);
+                    if (kvColumnsByEncodedColumnNames.put(pair, column)) {
+                        int count = 0;
+                        for (PColumn dupColumn : kvColumnsByEncodedColumnNames.get(pair)) {
+                            if (Objects.equal(familyName, dupColumn.getFamilyName())) {
+                                count++;
+                                if (count > 1) {
+                                    throw new ColumnAlreadyExistsException(schemaName.getString(), name.getString(), columnName);
+                                }
+                            }
                         }
                     }
                 }
@@ -518,7 +552,7 @@ public class PTableImpl implements PTable {
                 .orderedBy(Bytes.BYTES_COMPARATOR);
         for (int i = 0; i < families.length; i++) {
             Map.Entry<PName,List<PColumn>> entry = iterator.next();
-            PColumnFamily family = new PColumnFamilyImpl(entry.getKey(), entry.getValue());
+            PColumnFamily family = new PColumnFamilyImpl(entry.getKey(), entry.getValue(), EncodedColumnsUtil.usesEncodedColumnNames(storageScheme));
             families[i] = family;
             familyByString.put(family.getName().getString(), family);
             familyByBytes.put(family.getName().getBytes(), family);
@@ -544,9 +578,9 @@ public class PTableImpl implements PTable {
         for (PName name : this.physicalNames) {
             estimatedSize += name.getEstimatedSize();
         }
-
         this.estimatedSize = estimatedSize;
         this.baseColumnCount = baseColumnCount;
+        this.encodedCQCounter = encodedCQCounter;
     }
 
     @Override
@@ -736,7 +770,7 @@ public class PTableImpl implements PTable {
     }
 
     @Override
-    public PColumn getColumn(String name) throws ColumnNotFoundException, AmbiguousColumnException {
+    public PColumn getPColumnForColumnName(String name) throws ColumnNotFoundException, AmbiguousColumnException {
         List<PColumn> columns = columnsByName.get(name);
         int size = columns.size();
         if (size == 0) {
@@ -755,6 +789,38 @@ public class PTableImpl implements PTable {
         }
         return columns.get(0);
     }
+    
+    @Override
+    public PColumn getPColumnForColumnQualifier(byte[] cf, byte[] cq) throws ColumnNotFoundException, AmbiguousColumnException {
+        Preconditions.checkNotNull(cq);
+        if (!EncodedColumnsUtil.usesEncodedColumnNames(this) || cf == null) {
+            String columnName = (String)PVarchar.INSTANCE.toObject(cq);
+            return getPColumnForColumnName(columnName);
+        } else {
+            Integer qualifier = getEncodedColumnQualifier(cq);
+            String family = (String)PVarchar.INSTANCE.toObject(cf);
+            List<PColumn> columns = kvColumnsByEncodedColumnNames.get(new Pair<>(family, qualifier));
+            int size = columns.size();
+            if (size == 0) {
+                //TODO: samarth should we have a column qualifier not found exception?
+                throw new ColumnNotFoundException(Bytes.toString(cq));
+            }
+            //TODO: samarth I am not convinced if need this logic.
+//            if (size > 1) {
+//                for (PColumn column : columns) {
+//                    if (QueryConstants.DEFAULT_COLUMN_FAMILY.equals(column.getFamilyName().getString())) {
+//                        // Allow ambiguity with PK column or column in the default column family,
+//                        // since a PK column cannot be prefixed and a user would not know how to
+//                        // prefix a column in the default column family.
+//                        return column;
+//                    }
+//                }
+//                //TODO: samarth should we have a column qualifier not found exception?
+//                throw new AmbiguousColumnException(columns.get(0).getName().getString());
+//            }
+            return columns.get(0);
+        }
+    }
 
     /**
      *
@@ -775,6 +841,8 @@ public class PTableImpl implements PTable {
         private Mutation deleteRow;
         private final long ts;
         private final boolean hasOnDupKey;
+        // map from column name to value 
+        private Map<PColumn, byte[]> columnToValueMap; 
 
         public PRowImpl(KeyValueBuilder kvBuilder, ImmutableBytesWritable key, long ts, Integer bucketNum, boolean hasOnDupKey) {
             this.kvBuilder = kvBuilder;
@@ -787,7 +855,7 @@ public class PTableImpl implements PTable {
                 this.keyPtr =  new ImmutableBytesPtr(key);
                 this.key = ByteUtil.copyKeyBytesIfNecessary(key);
             }
-
+            this.columnToValueMap = Maps.newHashMapWithExpectedSize(1);//TODO: samarth size it properly
             newMutations();
         }
 
@@ -809,13 +877,49 @@ public class PTableImpl implements PTable {
                 // Include only deleteRow mutation if present because it takes precedence over all others
                 mutations.add(deleteRow);
             } else {
+                // store all columns for a given column family in a single cell instead of one column per cell in order to improve write performance
+                if (storageScheme == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+                    Put put = new Put(this.key);
+                    if (isWALDisabled()) {
+                        put.setDurability(Durability.SKIP_WAL);
+                    }
+                    // the setValues Put contains one cell per column, we need to convert it to a Put that contains a cell with all columns for a given column family
+                    for (PColumnFamily family : families) {
+                        byte[] columnFamily = family.getName().getBytes();
+                        Collection<PColumn> columns = family.getColumns();
+                        int maxEncodedColumnQualifier = Integer.MIN_VALUE;
+                        for (PColumn column : columns) {
+                            maxEncodedColumnQualifier = Math.max(maxEncodedColumnQualifier, column.getEncodedColumnQualifier());
+                        }
+                        byte[][] colValues = new byte[maxEncodedColumnQualifier+1][];
+                        for (PColumn column : columns) {
+                            colValues[column.getEncodedColumnQualifier()] = columnToValueMap.get(column);
+                        }
+                        
+                        List<Expression> children = Lists.newArrayListWithExpectedSize(columns.size());
+                        // create an expression list with all the columns
+                        for (int i=0; i<colValues.length; ++i) {
+                            children.add(new LiteralExpression(colValues[i]==null ? ByteUtil.EMPTY_BYTE_ARRAY : colValues[i] ));
+                        }
+                        // we use ArrayConstructorExpression to serialize multiple columns into a single byte[]
+                        // construct the ArrayConstructorExpression with a variable length data type since columns can be of fixed or variable length 
+                        ArrayConstructorExpression arrayExpression = new ArrayConstructorExpression(children, PVarbinary.INSTANCE, rowKeyOrderOptimizable);
+                        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                        arrayExpression.evaluate(new BaseTuple() {}, ptr);
+                        ImmutableBytesPtr colFamilyPtr = new ImmutableBytesPtr(columnFamily);
+                        addQuietly(put, kvBuilder, kvBuilder.buildPut(keyPtr,
+                            colFamilyPtr, colFamilyPtr, ts, ptr));
+                    }
+                    setValues = put;
+                }
                 // Because we cannot enforce a not null constraint on a KV column (since we don't know if the row exists when
-                // we upsert it), se instead add a KV that is always emtpy. This allows us to imitate SQL semantics given the
+                // we upsert it), so instead add a KV that is always empty. This allows us to imitate SQL semantics given the
                 // way HBase works.
+                Pair<byte[], byte[]> emptyKvInfo = EncodedColumnsUtil.getEmptyKeyValueInfo(PTableImpl.this);
                 addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
                     SchemaUtil.getEmptyColumnFamilyPtr(PTableImpl.this),
-                    QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts,
-                    QueryConstants.EMPTY_COLUMN_VALUE_BYTES_PTR));
+                    new ImmutableBytesPtr(emptyKvInfo.getFirst()), ts,
+                    new ImmutableBytesPtr(emptyKvInfo.getSecond())));
                 mutations.add(setValues);
                 if (!unsetValues.isEmpty()) {
                     mutations.add(unsetValues);
@@ -844,7 +948,8 @@ public class PTableImpl implements PTable {
         public void setValue(PColumn column, byte[] byteValue) {
             deleteRow = null;
             byte[] family = column.getFamilyName().getBytes();
-            byte[] qualifier = column.getName().getBytes();
+            byte[] qualifier = getColumnQualifier(column);
+            ImmutableBytesPtr qualifierPtr = new ImmutableBytesPtr(qualifier);
             PDataType<?> type = column.getDataType();
             // Check null, since some types have no byte representation for null
             if (byteValue == null) {
@@ -864,7 +969,7 @@ public class PTableImpl implements PTable {
                 // case of updates occurring due to the execution of the clause.
                 removeIfPresent(setValues, family, qualifier);
                 deleteQuietly(unsetValues, kvBuilder, kvBuilder.buildDeleteColumns(keyPtr, column
-                            .getFamilyName().getBytesPtr(), column.getName().getBytesPtr(), ts));
+                            .getFamilyName().getBytesPtr(), qualifierPtr, ts));
             } else {
                 ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue);
                 Integer	maxLength = column.getMaxLength();
@@ -877,9 +982,17 @@ public class PTableImpl implements PTable {
                 ptr.set(byteValue);
                 type.pad(ptr, maxLength, sortOrder);
                 removeIfPresent(unsetValues, family, qualifier);
-                addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
-                        column.getFamilyName().getBytesPtr(), column.getName().getBytesPtr(),
+             // store all columns for a given column family in a single cell instead of one column per cell in order to improve write performance
+                // we don't need to do anything with unsetValues as it is only used when storeNulls is false, storeNulls is always true when storeColsInSingleCell is true
+                if (storageScheme == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+                    columnToValueMap.put(column, ptr.get());
+                }
+                else {
+                    removeIfPresent(unsetValues, family, qualifier);
+                    addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
+                        column.getFamilyName().getBytesPtr(), qualifierPtr,
                         ts, ptr));
+                }
             }
         }
 
@@ -912,6 +1025,11 @@ public class PTableImpl implements PTable {
                 deleteRow.setDurability(Durability.SKIP_WAL);
             }
         }
+        
+        private byte[] getColumnQualifier(PColumn column) {
+            return EncodedColumnsUtil.getColumnQualifier(column, PTableImpl.this);
+        }
+        
     }
 
     @Override
@@ -1072,116 +1190,126 @@ public class PTableImpl implements PTable {
     public IndexType getIndexType() {
         return indexType;
     }
-
+    
     /**
      * Construct a PTable instance from ProtoBuffered PTable instance
      * @param table
      */
     public static PTable createFromProto(PTableProtos.PTable table) {
-      PName tenantId = null;
-      if(table.hasTenantId()){
-        tenantId = PNameFactory.newName(table.getTenantId().toByteArray());
-      }
-      PName schemaName = PNameFactory.newName(table.getSchemaNameBytes().toByteArray());
-      PName tableName = PNameFactory.newName(table.getTableNameBytes().toByteArray());
-      PTableType tableType = PTableType.values()[table.getTableType().ordinal()];
-      PIndexState indexState = null;
-      if (table.hasIndexState()) {
-        indexState = PIndexState.fromSerializedValue(table.getIndexState());
-      }
-      Short viewIndexId = null;
-      if(table.hasViewIndexId()){
-    	  viewIndexId = (short)table.getViewIndexId();
-      }
-      IndexType indexType = IndexType.getDefault();
-      if(table.hasIndexType()){
-          indexType = IndexType.fromSerializedValue(table.getIndexType().toByteArray()[0]);
-      }
-      long sequenceNumber = table.getSequenceNumber();
-      long timeStamp = table.getTimeStamp();
-      long indexDisableTimestamp = table.getIndexDisableTimestamp();
-      PName pkName = null;
-      if (table.hasPkNameBytes()) {
-        pkName = PNameFactory.newName(table.getPkNameBytes().toByteArray());
-      }
-      int bucketNum = table.getBucketNum();
-      List<PColumn> columns = Lists.newArrayListWithExpectedSize(table.getColumnsCount());
-      for (PTableProtos.PColumn curPColumnProto : table.getColumnsList()) {
-        columns.add(PColumnImpl.createFromProto(curPColumnProto));
-      }
-      List<PTable> indexes = Lists.newArrayListWithExpectedSize(table.getIndexesCount());
-      for (PTableProtos.PTable curPTableProto : table.getIndexesList()) {
-        indexes.add(createFromProto(curPTableProto));
-      }
+        PName tenantId = null;
+        if(table.hasTenantId()){
+            tenantId = PNameFactory.newName(table.getTenantId().toByteArray());
+        }
+        PName schemaName = PNameFactory.newName(table.getSchemaNameBytes().toByteArray());
+        PName tableName = PNameFactory.newName(table.getTableNameBytes().toByteArray());
+        PTableType tableType = PTableType.values()[table.getTableType().ordinal()];
+        PIndexState indexState = null;
+        if (table.hasIndexState()) {
+            indexState = PIndexState.fromSerializedValue(table.getIndexState());
+        }
+        Short viewIndexId = null;
+        if(table.hasViewIndexId()){
+            viewIndexId = (short)table.getViewIndexId();
+        }
+        IndexType indexType = IndexType.getDefault();
+        if(table.hasIndexType()){
+            indexType = IndexType.fromSerializedValue(table.getIndexType().toByteArray()[0]);
+        }
+        long sequenceNumber = table.getSequenceNumber();
+        long timeStamp = table.getTimeStamp();
+        long indexDisableTimestamp = table.getIndexDisableTimestamp();
+        PName pkName = null;
+        if (table.hasPkNameBytes()) {
+            pkName = PNameFactory.newName(table.getPkNameBytes().toByteArray());
+        }
+        int bucketNum = table.getBucketNum();
+        List<PColumn> columns = Lists.newArrayListWithExpectedSize(table.getColumnsCount());
+        for (PTableProtos.PColumn curPColumnProto : table.getColumnsList()) {
+            columns.add(PColumnImpl.createFromProto(curPColumnProto));
+        }
+        List<PTable> indexes = Lists.newArrayListWithExpectedSize(table.getIndexesCount());
+        for (PTableProtos.PTable curPTableProto : table.getIndexesList()) {
+            indexes.add(createFromProto(curPTableProto));
+        }
 
-      boolean isImmutableRows = table.getIsImmutableRows();
-      PName parentSchemaName = null;
-      PName parentTableName = null;
-      if (table.hasParentNameBytes()) {
-        parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName((table.getParentNameBytes().toByteArray())));
-        parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(table.getParentNameBytes().toByteArray()));
-      }
-      PName defaultFamilyName = null;
-      if (table.hasDefaultFamilyName()) {
-        defaultFamilyName = PNameFactory.newName(table.getDefaultFamilyName().toByteArray());
-      }
-      boolean disableWAL = table.getDisableWAL();
-      boolean multiTenant = table.getMultiTenant();
-      boolean storeNulls = table.getStoreNulls();
-      boolean isTransactional = table.getTransactional();
-      ViewType viewType = null;
-      String viewStatement = null;
-      List<PName> physicalNames = Collections.emptyList();
-      if (tableType == PTableType.VIEW) {
-        viewType = ViewType.fromSerializedValue(table.getViewType().toByteArray()[0]);
-      }
-      if(table.hasViewStatement()){
-        viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
-      }
-      if (tableType == PTableType.VIEW || viewIndexId != null) {
-        physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
-        for(int i = 0; i < table.getPhysicalNamesCount(); i++){
-          physicalNames.add(PNameFactory.newName(table.getPhysicalNames(i).toByteArray()));
+        boolean isImmutableRows = table.getIsImmutableRows();
+        PName parentSchemaName = null;
+        PName parentTableName = null;
+        if (table.hasParentNameBytes()) {
+            parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName((table.getParentNameBytes().toByteArray())));
+            parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(table.getParentNameBytes().toByteArray()));
+        }
+        PName defaultFamilyName = null;
+        if (table.hasDefaultFamilyName()) {
+            defaultFamilyName = PNameFactory.newName(table.getDefaultFamilyName().toByteArray());
+        }
+        boolean disableWAL = table.getDisableWAL();
+        boolean multiTenant = table.getMultiTenant();
+        boolean storeNulls = table.getStoreNulls();
+        boolean isTransactional = table.getTransactional();
+        ViewType viewType = null;
+        String viewStatement = null;
+        List<PName> physicalNames = Collections.emptyList();
+        if (tableType == PTableType.VIEW) {
+            viewType = ViewType.fromSerializedValue(table.getViewType().toByteArray()[0]);
+        }
+        if(table.hasViewStatement()){
+            viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
+        }
+        if (tableType == PTableType.VIEW || viewIndexId != null) {
+            physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
+            for(int i = 0; i < table.getPhysicalNamesCount(); i++) {
+                physicalNames.add(PNameFactory.newName(table.getPhysicalNames(i).toByteArray()));
+            }
+        }
+        int baseColumnCount = -1;
+        if (table.hasBaseColumnCount()) {
+            baseColumnCount = table.getBaseColumnCount();
         }
-      }
-      
-      int baseColumnCount = -1;
-      if (table.hasBaseColumnCount()) {
-          baseColumnCount = table.getBaseColumnCount();
-      }
 
-      boolean rowKeyOrderOptimizable = false;
-      if (table.hasRowKeyOrderOptimizable()) {
-          rowKeyOrderOptimizable = table.getRowKeyOrderOptimizable();
-      }
-      long updateCacheFrequency = 0;
-      if (table.hasUpdateCacheFrequency()) {
-          updateCacheFrequency = table.getUpdateCacheFrequency();
-      }
-      boolean isNamespaceMapped=false;
-      if (table.hasIsNamespaceMapped()) {
-          isNamespaceMapped = table.getIsNamespaceMapped();
-      }
-      String autoParititonSeqName = null;
-      if (table.hasAutoParititonSeqName()) {
-          autoParititonSeqName = table.getAutoParititonSeqName();
-      }
-      boolean isAppendOnlySchema = false;
-      if (table.hasIsAppendOnlySchema()) {
-          isAppendOnlySchema = table.getIsAppendOnlySchema();
-      }
-      
-      try {
-        PTableImpl result = new PTableImpl();
-        result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
-            (bucketNum == NO_SALTING) ? null : bucketNum, columns, parentSchemaName, parentTableName, indexes,
-            isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
-            multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-            isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, isAppendOnlySchema);
-        return result;
-      } catch (SQLException e) {
-        throw new RuntimeException(e); // Impossible
-      }
+        boolean rowKeyOrderOptimizable = false;
+        if (table.hasRowKeyOrderOptimizable()) {
+            rowKeyOrderOptimizable = table.getRowKeyOrderOptimizable();
+        }
+        long updateCacheFrequency = 0;
+        if (table.hasUpdateCacheFrequency()) {
+            updateCacheFrequency = table.getUpdateCacheFrequency();
+        }
+        boolean isNamespaceMapped=false;
+        if (table.hasIsNamespaceMapped()) {
+            isNamespaceMapped = table.getIsNamespaceMapped();
+        }
+        String autoParititonSeqName = null;
+        if (table.hasAutoParititonSeqName()) {
+            autoParititonSeqName = table.getAutoParititonSeqName();
+        }
+        boolean isAppendOnlySchema = false;
+        if (table.hasIsAppendOnlySchema()) {
+            isAppendOnlySchema = table.getIsAppendOnlySchema();
+        }
+        StorageScheme storageScheme = null;
+        if (table.hasStorageScheme()) {
+            storageScheme = StorageScheme.fromSerializedValue(table.getStorageScheme().toByteArray()[0]);
+        }
+        EncodedCQCounter encodedColumnQualifierCounter = EncodedColumnsUtil.usesEncodedColumnNames(storageScheme) ? new EncodedCQCounter() : EncodedCQCounter.NULL_COUNTER;
+        if (table.getEncodedCQCountersList() != null) {
+            encodedColumnQualifierCounter = new EncodedCQCounter();
+            for (org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter cqCounterFromProto : table.getEncodedCQCountersList()) {
+                encodedColumnQualifierCounter.setValue(cqCounterFromProto.getColFamily(), cqCounterFromProto.getCounter());
+            }
+        }
+
+        try {
+            PTableImpl result = new PTableImpl();
+            result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
+                (bucketNum == NO_SALTING) ? null : bucketNum, columns, parentSchemaName, parentTableName, indexes,
+                        isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
+                        multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
+                        isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, isAppendOnlySchema, storageScheme, encodedColumnQualifierCounter);
+            return result;
+        } catch (SQLException e) {
+            throw new RuntimeException(e); // Impossible
+        }
     }
 
     public static PTableProtos.PTable toProto(PTable table) {
@@ -1259,10 +1387,22 @@ public class PTableImpl implements PTable {
       builder.setUpdateCacheFrequency(table.getUpdateCacheFrequency());
       builder.setIndexDisableTimestamp(table.getIndexDisableTimestamp());
       builder.setIsNamespaceMapped(table.isNamespaceMapped());
-      if (table.getAutoPartitionSeqName()!= null) {
+      if (table.getAutoPartitionSeqName() != null) {
           builder.setAutoParititonSeqName(table.getAutoPartitionSeqName());
       }
       builder.setIsAppendOnlySchema(table.isAppendOnlySchema());
+      if (table.getStorageScheme() != null) {
+          builder.setStorageScheme(ByteStringer.wrap(new byte[]{table.getStorageScheme().getSerializedValue()}));
+      }
+      if (table.getEncodedCQCounter() != null) {
+          Map<String, Integer> values = table.getEncodedCQCounter().values();
+          for (Entry<String, Integer> cqCounter : values.entrySet()) {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder cqBuilder = org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.newBuilder();
+              cqBuilder.setColFamily(cqCounter.getKey());
+              cqBuilder.setCounter(cqCounter.getValue());
+              builder.addEncodedCQCounters(cqBuilder.build());
+          }
+      }
       return builder.build();
     }
 
@@ -1332,4 +1472,14 @@ public class PTableImpl implements PTable {
         } else if (!key.equals(other.getKey())) return false;
         return true;
     }
+    
+    @Override
+    public StorageScheme getStorageScheme() {
+        return storageScheme;
+    }
+    
+    @Override
+    public EncodedCQCounter getEncodedCQCounter() {
+        return encodedCQCounter;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
index 42699d9..017c75d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableKey.java
@@ -28,7 +28,11 @@ public class PTableKey {
     public PTableKey(PName tenantId, String name) {
         Preconditions.checkNotNull(name);
         this.tenantId = tenantId;
-        this.name = name;
+        if (name.indexOf(QueryConstants.NAMESPACE_SEPARATOR) != -1) {
+            this.name = name.replace(QueryConstants.NAMESPACE_SEPARATOR, QueryConstants.NAME_SEPARATOR);
+        } else {
+            this.name = name;
+        }
     }
 
     public PName getTenantId() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/ProjectedColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ProjectedColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ProjectedColumn.java
index 19dd1c1..9336938 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ProjectedColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ProjectedColumn.java
@@ -39,6 +39,7 @@ public class ProjectedColumn extends DelegateColumn {
         return name;
     }
     
+    @Override
     public PName getFamilyName() {
         return familyName;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
index 734a9ed..23cfd1b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/SaltingUtil.java
@@ -38,7 +38,7 @@ public class SaltingUtil {
     public static final String SALTING_COLUMN_NAME = "_SALT";
     public static final String SALTED_ROW_KEY_NAME = "_SALTED_KEY";
     public static final PColumnImpl SALTING_COLUMN = new PColumnImpl(
-            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false);
+            PNameFactory.newName(SALTING_COLUMN_NAME), null, PBinary.INSTANCE, 1, 0, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null);
     public static final RowKeySchema VAR_BINARY_SALTED_SCHEMA = new RowKeySchemaBuilder(2)
         .addField(SALTING_COLUMN, false, SortOrder.getDefault())
         .addField(SchemaUtil.VAR_BINARY_DATUM, false, SortOrder.getDefault()).build();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
index 26a7718..bab7231 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/TableProperty.java
@@ -51,7 +51,7 @@ public enum TableProperty {
     STORE_NULLS(PhoenixDatabaseMetaData.STORE_NULLS, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false),
     
     TRANSACTIONAL(PhoenixDatabaseMetaData.TRANSACTIONAL, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false),
-
+    
     UPDATE_CACHE_FREQUENCY(PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY, true, true) {
 	    @Override
         public Object getValue(Object value) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/BaseTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/BaseTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/BaseTuple.java
index a8dc487..8028eb2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/BaseTuple.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/BaseTuple.java
@@ -17,11 +17,50 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
 
 public abstract class BaseTuple implements Tuple {
+    @Override
+    public int size() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public boolean isImmutable() {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public void getKey(ImmutableBytesWritable ptr) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Cell getValue(int index) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public Cell getValue(byte [] family, byte [] qualifier) {
+        throw new UnsupportedOperationException();
+    }
+    
+    @Override
+    public boolean getValue(byte [] family, byte [] qualifier, ImmutableBytesWritable ptr) {
+        throw new UnsupportedOperationException();
+    }
 
     @Override
     public long getSequenceValue(int index) {
         throw new UnsupportedOperationException();
     }
+    
+    @Override
+    public void setKeyValues(List<Cell> values) {
+        throw new UnsupportedOperationException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/DelegateTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/DelegateTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/DelegateTuple.java
index 58b1eda..3430f5b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/DelegateTuple.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/DelegateTuple.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import java.util.List;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 
@@ -61,4 +63,9 @@ public class DelegateTuple implements Tuple {
     public long getSequenceValue(int index) {
         return delegate.getSequenceValue(index);
     }
+
+    @Override
+    public void setKeyValues(List<Cell> values) {
+        delegate.setKeyValues(values);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/EncodedColumnQualiferCellsList.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/EncodedColumnQualiferCellsList.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/EncodedColumnQualiferCellsList.java
new file mode 100644
index 0000000..f39bb1f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/EncodedColumnQualiferCellsList.java
@@ -0,0 +1,569 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.tuple;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_EMPTY_COLUMN_NAME;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
+
+import java.util.Collection;
+import java.util.ConcurrentModificationException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.phoenix.schema.PTable.StorageScheme;
+
+/**
+ * List implementation that provides indexed based look up when the cell column qualifiers are positive numbers. 
+ * These qualifiers are generated by using one of the column qualifier encoding schemes specified in {@link StorageScheme}. 
+ * The api methods in this list assume that the caller wants to see
+ * and add only non null elements in the list. 
+ * <p>
+ * Please note that this implementation doesn't implement all the optional methods of the 
+ * {@link List} interface. Such unsupported methods could violate the basic invariance of the list that every cell with
+ * an encoded column qualifier has a fixed position in the list.
+ * </p>
+ * <p>
+ * An important performance characteristic of this list is that doing look up on the basis of index via {@link #get(int)}
+ * is an O(n) operation. This makes iterating through the list using {@link #get(int)} an O(n^2) operation.
+ * Instead, for iterating through the list, one should use the iterators created through {@link #iterator()} or 
+ * {@link #listIterator()}. Do note that getting an element using {@link #getCellForColumnQualifier(int)} is an O(1) operation
+ * and should generally be the way for getting elements out of the list.
+ * </p> 
+ */
+@NotThreadSafe
+public class EncodedColumnQualiferCellsList implements List<Cell> {
+
+    private int minQualifier;
+    private int maxQualifier;
+    private int nonReservedRangeOffset;
+    private final Cell[] array;
+    private int numNonNullElements;
+    private int firstNonNullElementIdx = -1;
+    private static final int RESERVED_RANGE_SIZE = ENCODED_CQ_COUNTER_INITIAL_VALUE - ENCODED_EMPTY_COLUMN_NAME;
+    // Used by iterators to figure out if the list was structurally modified.
+    private int modCount = 0;
+
+    public EncodedColumnQualiferCellsList(int minQ, int maxQ) {
+        checkArgument(minQ <= maxQ, "Invalid arguments. Min: " + minQ
+                + ". Max: " + maxQ);
+        this.minQualifier = minQ;
+        this.maxQualifier = maxQ;
+        int size = 0;
+        if (maxQ < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            size = RESERVED_RANGE_SIZE;
+        } else if (minQ < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            size = (maxQ - minQ + 1);
+        } else {
+            size = RESERVED_RANGE_SIZE + (maxQ - minQ + 1);
+        }
+        this.array = new Cell[size];
+        this.nonReservedRangeOffset = minQ > ENCODED_CQ_COUNTER_INITIAL_VALUE ? minQ  - ENCODED_CQ_COUNTER_INITIAL_VALUE : 0;
+    }
+
+    @Override
+    public int size() {
+        return numNonNullElements;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return numNonNullElements == 0;
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return indexOf(o) >= 0;
+    }
+
+    @Override
+    public Object[] toArray() {
+        Object[] toReturn = new Object[numNonNullElements];
+        int counter = 0;
+        if (numNonNullElements > 0) {
+            for (int i = 0; i < array.length; i++) {
+                if (array[i] != null) {
+                    toReturn[counter++] = array[i];
+                }
+            }
+        }
+        return toReturn;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> T[] toArray(T[] a) {
+        T[] toReturn =
+                (T[]) java.lang.reflect.Array.newInstance(a.getClass().getComponentType(),
+                    numNonNullElements);
+        int counter = 0;
+        for (int i = 0; i < array.length; i++) {
+            if (array[i] != null) {
+                toReturn[counter++] = (T) array[i];
+            }
+        }
+        return toReturn;
+    }
+
+    @Override
+    public boolean add(Cell e) {
+        if (e == null) {
+            throw new NullPointerException();
+        }
+        int columnQualifier = getEncodedColumnQualifier(e.getQualifierArray(), e.getQualifierOffset(), e.getQualifierLength());
+                
+        checkQualifierRange(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        if (array[idx] == null) {
+            numNonNullElements++;
+        }
+        array[idx] = e;
+        if (firstNonNullElementIdx == -1) {
+            firstNonNullElementIdx = idx;
+        } else if (idx < firstNonNullElementIdx) {
+            firstNonNullElementIdx = idx;
+        }
+        modCount++;
+        /*
+         * Note that we don't care about equality of the element being added with the element
+         * already present at the index.
+         */
+        return true;
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        if (o == null) {
+            return false;
+        }
+        Cell e = (Cell) o;
+        int i = 0;
+        while (i < array.length) {
+            if (array[i] != null && array[i].equals(e)) {
+                array[i] = null;
+                numNonNullElements--;
+                if (numNonNullElements == 0) {
+                    firstNonNullElementIdx = -1;
+                } else if (firstNonNullElementIdx == i) {
+                    // the element being removed was the first non-null element we knew
+                    while (i < array.length && (array[i]) == null) {
+                        i++;
+                    }
+                    if (i < array.length) {
+                        firstNonNullElementIdx = i;
+                    } else {
+                        firstNonNullElementIdx = -1;
+                    }
+                }
+                modCount++;
+                return true;
+            }
+            i++;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean containsAll(Collection<?> c) {
+        boolean containsAll = true;
+        Iterator<?> itr = c.iterator();
+        while (itr.hasNext()) {
+            containsAll &= (indexOf(itr.next()) >= 0);
+        }
+        return containsAll;
+    }
+
+    @Override
+    public boolean addAll(Collection<? extends Cell> c) {
+        boolean changed = false;
+        for (Cell cell : c) {
+            if (c == null) {
+                throw new NullPointerException();
+            }
+            changed |= add(cell);
+        }
+        return changed;
+    }
+
+    @Override
+    public boolean addAll(int index, Collection<? extends Cell> c) {
+        throwGenericUnsupportedOperationException();
+        return false;
+    }
+
+    @Override
+    public boolean removeAll(Collection<?> c) {
+        Iterator<?> itr = c.iterator();
+        boolean changed = false;
+        while (itr.hasNext()) {
+            changed |= remove(itr.next());
+        }
+        return changed;
+    }
+
+    @Override
+    public boolean retainAll(Collection<?> collection) {
+        boolean changed = false;
+        // Optimize if the passed collection is an instance of EncodedColumnQualiferCellsList
+        if (collection instanceof EncodedColumnQualiferCellsList) {
+            EncodedColumnQualiferCellsList list = (EncodedColumnQualiferCellsList) collection;
+            ListIterator<Cell> listItr = this.listIterator();
+            while (listItr.hasNext()) {
+                Cell cellInThis = listItr.next();
+                int qualifier = getEncodedColumnQualifier(cellInThis.getQualifierArray(),
+                            cellInThis.getQualifierOffset(), cellInThis.getQualifierLength());
+                try {
+                    Cell cellInParam = list.getCellForColumnQualifier(qualifier);
+                    if (cellInParam != null && cellInParam.equals(cellInThis)) {
+                        continue;
+                    }
+                    listItr.remove();
+                    changed = true;
+                } catch (IndexOutOfBoundsException expected) {
+                    // this could happen when the qualifier of cellInParam lies out of
+                    // the range of this list.
+                    listItr.remove();
+                    changed = true;
+                }
+            }
+        } else {
+            throw new UnsupportedOperationException(
+                    "Operation only supported for collections of type EncodedColumnQualiferCellsList");
+        }
+        return changed;
+    }
+
+    @Override
+    public void clear() {
+        for (int i = 0; i < array.length; i++) {
+            array[i] = null;
+        }
+        firstNonNullElementIdx = -1;
+        numNonNullElements = 0;
+        modCount++;
+    }
+
+    @Override
+    public Cell get(int index) {
+        rangeCheck(index);
+        int numNonNullElementsFound = 0;
+        for (int i = 0; i < array.length; i++) {
+            if (array[i] != null) {
+                numNonNullElementsFound++;
+                if (numNonNullElementsFound == index + 1) {
+                    return array[i];
+                }
+            }
+        }
+        throw new IllegalStateException("There was no element present in the list at index "
+                + index + " even though number of elements in the list are " + size());
+    }
+
+    @Override
+    public Cell set(int index, Cell e) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public void add(int index, Cell element) {
+        throwGenericUnsupportedOperationException();
+    }
+
+    @Override
+    public Cell remove(int index) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public int indexOf(Object o) {
+        if (o == null || isEmpty()) {
+            return -1;
+        } else {
+            int numNonNull = -1;
+            for (int i = 0; i < array.length; i++) {
+                if (array[i] != null) {
+                    numNonNull++;
+                }
+                if (o.equals(array[i])) {
+                    return numNonNull;
+                }
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public int lastIndexOf(Object o) {
+        if (o == null || isEmpty()) {
+            return -1;
+        }
+        int lastIndex = numNonNullElements;
+        for (int i = array.length - 1; i >= 0; i--) {
+            if (array[i] != null) {
+                lastIndex--;
+            }
+            if (o.equals(array[i])) {
+                return lastIndex;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator() {
+        return new ListItr();
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator(int index) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public List<Cell> subList(int fromIndex, int toIndex) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public Iterator<Cell> iterator() {
+        return new Itr();
+    }
+
+    public Cell getCellForColumnQualifier(int columnQualifier) {
+        checkQualifierRange(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        Cell c = array[idx];
+        return c;
+    }
+
+    public Cell getFirstCell() {
+        if (firstNonNullElementIdx == -1) {
+            throw new NoSuchElementException("No elements present in the list");
+        }
+        return array[firstNonNullElementIdx];
+    }
+
+    private void checkQualifierRange(int qualifier) {
+        if (qualifier < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            return; // space in the array for reserved range is always allocated. 
+        }
+        if (qualifier < minQualifier || qualifier > maxQualifier) {
+            throw new IndexOutOfBoundsException("Qualifier " + qualifier
+                    + " is out of the valid range - (" + minQualifier + ", " + maxQualifier + ")");
+        }
+    }
+
+    private void rangeCheck(int index) {
+        if (index < 0 || index >= size()) {
+            throw new IndexOutOfBoundsException();
+        }
+    }
+
+    private int getArrayIndex(int columnQualifier) {
+        checkArgument(columnQualifier >= ENCODED_EMPTY_COLUMN_NAME);
+        if (columnQualifier < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            return columnQualifier;
+        }
+        return columnQualifier - nonReservedRangeOffset;
+    }
+
+    private void throwGenericUnsupportedOperationException() {
+        throw new UnsupportedOperationException(
+                "Operation cannot be supported because it potentially violates the invariance contract of this list implementation");
+    }
+
+    private class Itr implements Iterator<Cell> {
+        protected int nextIndex = 0;
+        protected int lastRet = -1;
+        protected int expectedModCount = modCount;
+        
+        private Itr() {
+            moveForward(true);
+        }
+
+        @Override
+        public boolean hasNext() {
+            return nextIndex != -1;
+        }
+
+        @Override
+        public Cell next() {
+            checkForCoModification();
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            Cell next = array[nextIndex];
+            lastRet = nextIndex;
+            moveForward(false);
+            modCount++;
+            expectedModCount = modCount;
+            return next;
+        }
+
+        @Override
+        public void remove() {
+            if (lastRet < 0) {
+                throw new IllegalStateException();
+            }
+            checkForCoModification();
+            array[lastRet] = null;
+            lastRet = -1;
+            numNonNullElements--;
+            modCount++;
+            expectedModCount = modCount;
+        }
+
+        protected void moveForward(boolean init) {
+            int i = init ? 0 : nextIndex + 1;
+            while (i < array.length && (array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                nextIndex = i;
+            } else {
+                nextIndex = -1;
+            }
+        }
+        
+        protected void checkForCoModification() {
+            if (modCount != expectedModCount) {
+                throw new ConcurrentModificationException();
+            }
+        }
+
+    }
+
+    private class ListItr extends Itr implements ListIterator<Cell> {
+        private int previousIndex = -1;
+        
+        private ListItr() {
+            moveForward(true);
+        }
+
+        @Override
+        public boolean hasNext() {
+            return nextIndex != -1;
+        }
+
+        @Override
+        public boolean hasPrevious() {
+            return previousIndex != -1;
+        }
+
+        @Override
+        public Cell previous() {
+            if (previousIndex == -1) {
+                throw new NoSuchElementException();
+            }
+            checkForCoModification();
+            lastRet = previousIndex;
+            movePointersBackward();
+            return array[lastRet];
+        }
+
+        @Override
+        public int nextIndex() {
+            return nextIndex;
+        }
+
+        @Override
+        public int previousIndex() {
+            return previousIndex;
+        }
+
+        @Override
+        public void remove() {
+            if (lastRet == nextIndex) {
+                moveNextPointer(nextIndex);
+            }
+            super.remove();
+            expectedModCount = modCount;
+        }
+
+        @Override
+        public void set(Cell e) {
+            if (lastRet == -1) {
+                throw new IllegalStateException();
+            }
+            int columnQualifier = getEncodedColumnQualifier(e.getQualifierArray(), e.getQualifierOffset(), e.getQualifierLength());                    
+            int idx = getArrayIndex(columnQualifier);
+            if (idx != lastRet) {
+                throw new IllegalArgumentException("Cell " + e + " with column qualifier "
+                        + columnQualifier + " belongs at index " + idx
+                        + ". It cannot be added at the position " + lastRet
+                        + " to which the previous next() or previous() was pointing to.");
+            }
+            EncodedColumnQualiferCellsList.this.add(e);
+            expectedModCount = modCount;
+        }
+
+        @Override
+        public void add(Cell e) {
+            throwGenericUnsupportedOperationException();
+        }
+        
+        @Override
+        protected void moveForward(boolean init) {
+            if (!init) {
+                previousIndex = nextIndex;
+            }
+            int i = init ? 0 : nextIndex + 1; 
+            moveNextPointer(i);
+        }
+
+        private void moveNextPointer(int i) {
+            while (i < array.length && (array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                nextIndex = i;
+            } else {
+                nextIndex = -1;
+            }
+        }
+
+        private void movePointersBackward() {
+            nextIndex = previousIndex;
+            int i = previousIndex - 1;
+            movePreviousPointer(i);
+        }
+
+        private void movePreviousPointer(int i) {
+            for (; i >= 0; i--) {
+                if (array[i] != null) {
+                    previousIndex = i;
+                    break;
+                }
+            }
+            if (i < 0) {
+                previousIndex = -1;
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/MultiKeyValueTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/MultiKeyValueTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/MultiKeyValueTuple.java
index 53f155b..d946870 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/MultiKeyValueTuple.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/MultiKeyValueTuple.java
@@ -36,6 +36,7 @@ public class MultiKeyValueTuple extends BaseTuple {
     }
 
     /** Caller must not modify the list that is passed here */
+    @Override
     public void setKeyValues(List<Cell> values) {
         this.values = values;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedMultiKeyValueTuple.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedMultiKeyValueTuple.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedMultiKeyValueTuple.java
new file mode 100644
index 0000000..08cafe0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedMultiKeyValueTuple.java
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.schema.tuple;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
+
+import java.util.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+/**
+ * Tuple that uses the 
+ */
+public class PositionBasedMultiKeyValueTuple extends BaseTuple {
+    private EncodedColumnQualiferCellsList values;
+
+    public PositionBasedMultiKeyValueTuple() {}
+    
+    public PositionBasedMultiKeyValueTuple(List<Cell> values) {
+        checkArgument(values instanceof EncodedColumnQualiferCellsList, "PositionBasedMultiKeyValueTuple only works with lists of type EncodedColumnQualiferCellsList");
+        this.values = (EncodedColumnQualiferCellsList)values;
+    }
+    
+    /** Caller must not modify the list that is passed here */
+    @Override
+    public void setKeyValues(List<Cell> values) {
+        checkArgument(values instanceof EncodedColumnQualiferCellsList, "PositionBasedMultiKeyValueTuple only works with lists of type EncodedColumnQualiferCellsList");
+        this.values = (EncodedColumnQualiferCellsList)values;
+    }
+
+    @Override
+    public void getKey(ImmutableBytesWritable ptr) {
+        Cell value = values.getFirstCell();
+        ptr.set(value.getRowArray(), value.getRowOffset(), value.getRowLength());
+    }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
+
+    @Override
+    public Cell getValue(byte[] family, byte[] qualifier) {
+        return values.getCellForColumnQualifier(getEncodedColumnQualifier(qualifier));
+    }
+
+    @Override
+    public String toString() {
+        return values.toString();
+    }
+
+    @Override
+    public int size() {
+        return values.size();
+    }
+
+    @Override
+    public Cell getValue(int index) {
+        return values.get(index);
+    }
+
+    @Override
+    public boolean getValue(byte[] family, byte[] qualifier,
+            ImmutableBytesWritable ptr) {
+        Cell kv = getValue(family, qualifier);
+        if (kv == null)
+            return false;
+        ptr.set(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
+        return true;
+    }}


[02/13] phoenix git commit: PHOENIX-3463 ViewIndexIT#testUpdateOnTenantViewWithGlobalView doesn't test namespace mapping correctly

Posted by sa...@apache.org.
PHOENIX-3463 ViewIndexIT#testUpdateOnTenantViewWithGlobalView doesn't test namespace mapping correctly


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

Branch: refs/heads/encodecolumns2
Commit: b1063479ccdc2e6e088c00b34bf2704732da0e24
Parents: c030eed
Author: Samarth <sa...@salesforce.com>
Authored: Mon Nov 7 16:48:47 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Nov 7 16:48:47 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/end2end/index/ViewIndexIT.java  | 17 ++++++++++-------
 1 file changed, 10 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/b1063479/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
index 46aefff..25cb632 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ViewIndexIT.java
@@ -317,8 +317,7 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testUpdateOnTenantViewWithGlobalView() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
+        Connection conn = getConnection();
         String baseSchemaName = "PLATFORM_ENTITY";
         String baseTableName = generateUniqueName();
         String baseFullName = SchemaUtil.getTableName(baseSchemaName, baseTableName);
@@ -328,6 +327,9 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
         String tsViewTableName = "TSV_" + generateUniqueName();
         String tsViewFullName = SchemaUtil.getTableName(baseSchemaName, tsViewTableName);
         try {
+            if (isNamespaceMapped) {
+                conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + baseSchemaName);
+            }
             conn.createStatement().execute(
                     "CREATE TABLE " + baseFullName + "(\n" + "    ORGANIZATION_ID CHAR(15) NOT NULL,\n"
                             + "    KEY_PREFIX CHAR(3) NOT NULL,\n" + "    CREATED_DATE DATE,\n"
@@ -360,21 +362,22 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
             tsConn.createStatement().execute("UPSERT INTO " + tsViewFullName + "(INT1,DOUBLE1,IS_BOOLEAN,TEXT1) VALUES (10,10.0, true, 'j')");
             tsConn.commit();
             
-            assertRowCount(tsConn, tsViewFullName, baseFullName, 10);
+            String basePhysicalName = SchemaUtil.getPhysicalHBaseTableName(baseFullName, isNamespaceMapped, PTableType.TABLE).getString();
+            assertRowCount(tsConn, tsViewFullName, basePhysicalName, 10);
             
             tsConn.createStatement().execute("DELETE FROM " + tsViewFullName + " WHERE TEXT1='d'");
             tsConn.commit();
-            assertRowCount(tsConn, tsViewFullName, baseFullName, 9);
+            assertRowCount(tsConn, tsViewFullName, basePhysicalName, 9);
 
             tsConn.createStatement().execute("DELETE FROM " + tsViewFullName + " WHERE INT1=2");
             tsConn.commit();
-            assertRowCount(tsConn, tsViewFullName, baseFullName, 8);
+            assertRowCount(tsConn, tsViewFullName, basePhysicalName, 8);
             
             // Use different connection for delete
             Connection tsConn2 = DriverManager.getConnection(getUrl(), tsProps);
             tsConn2.createStatement().execute("DELETE FROM " + tsViewFullName + " WHERE DOUBLE1 > 7.5 AND DOUBLE1 < 9.5");
             tsConn2.commit();
-            assertRowCount(tsConn2, tsViewFullName, baseFullName, 6);
+            assertRowCount(tsConn2, tsViewFullName, basePhysicalName, 6);
             
             tsConn2.createStatement().execute("DROP VIEW " + tsViewFullName);
             // Should drop view and index and remove index data
@@ -400,7 +403,7 @@ public class ViewIndexIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute(
                     "CREATE INDEX " + indexName + " \n" + "ON " + viewFullName + " (TEXT1 DESC, INT1)\n"
                             + "INCLUDE (CREATED_BY, DOUBLE1, IS_BOOLEAN, CREATED_DATE)");
-            assertRowCount(tsConn3, tsViewFullName, baseFullName, 0);
+            assertRowCount(tsConn3, tsViewFullName, basePhysicalName, 0);
             
             tsConn.close();
             tsConn2.close();


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java
new file mode 100644
index 0000000..564e75e
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/EncodedColumnQualifierCellsListTest.java
@@ -0,0 +1,608 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.query;
+
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.ConcurrentModificationException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.tuple.EncodedColumnQualiferCellsList;
+import org.junit.Test;
+
+public class EncodedColumnQualifierCellsListTest {
+    
+    private static final byte[] row = Bytes.toBytes("row");
+    private static final byte[] cf = Bytes.toBytes("cf");
+
+    
+    @Test
+    public void testIterator() {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        int i = 0;
+        populateListAndArray(list, cells);
+        Iterator itr = list.iterator();
+        assertTrue(itr.hasNext());
+        
+        // test itr.next()
+        i = 0;
+        while (itr.hasNext()) {
+            assertEquals(cells[i++], itr.next());
+        }
+        
+        assertEquals(7, list.size());
+        
+        // test itr.remove()
+        itr = list.iterator();
+        i = 0;
+        int numRemoved = 0;
+        try {
+            itr.remove();
+            fail("Remove not allowed till next() is called");
+        } catch (IllegalStateException expected) {}
+        
+        while (itr.hasNext()) {
+            assertEquals(cells[i++], itr.next());
+            itr.remove();
+            numRemoved++;
+        }
+        assertEquals("Number of elements removed should have been the size of the list", 7, numRemoved);
+    }
+    
+    @Test
+    public void testSize() {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        assertEquals(0, list.size());
+        
+        populateList(list);
+        
+        assertEquals(7, list.size());
+        int originalSize = list.size();
+        
+        Iterator itr = list.iterator();
+        while (itr.hasNext()) {
+            itr.next();
+            itr.remove();
+            assertEquals(--originalSize, list.size());
+        }
+    }
+    
+    @Test
+    public void testIsEmpty() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        assertTrue(list.isEmpty());
+        populateList(list);
+        assertFalse(list.isEmpty());
+        Iterator itr = list.iterator();
+        while (itr.hasNext()) {
+            itr.next();
+            itr.remove();
+            if (itr.hasNext()) {
+                assertFalse(list.isEmpty());
+            }
+        }
+        assertTrue(list.isEmpty());
+    }
+    
+    @Test
+    public void testContains() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        
+        for (Cell c : cells) {
+            assertTrue(list.contains(c));
+        }
+        assertFalse(list.contains(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(13))));
+    }
+    
+    @Test
+    public void testToArrayWithParam() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        Cell[] array = list.toArray(new Cell[0]);
+        assertTrue(Arrays.equals(cells, array));
+    }
+    
+    @Test
+    public void testToArrayWithoutParam() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        Object[] array = list.toArray();
+        assertTrue(Arrays.equals(cells, array));
+    }
+    
+    @Test
+    public void testRemove() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        assertTrue(list.remove(cells[0]));
+        assertEquals(6, list.size());
+        assertTrue(list.remove(cells[6]));
+        assertEquals(5, list.size());
+        assertTrue(list.remove(cells[3]));
+        assertEquals(4, list.size());
+        assertFalse(list.remove(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(13))));
+        assertEquals(4, list.size());
+    }
+    
+    @Test
+    public void testContainsAll() throws Exception {
+        EncodedColumnQualiferCellsList list1 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list1);
+        EncodedColumnQualiferCellsList list2 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list2);
+        assertTrue(list1.containsAll(list2));
+        list2.remove(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(11)));
+        assertTrue(list1.containsAll(list2));
+        assertFalse(list2.containsAll(list1));
+        list2.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(13)));
+        assertFalse(list1.containsAll(list2));
+        assertFalse(list2.containsAll(list1));
+        List<Cell> arrayList = new ArrayList<>();
+        populateList(arrayList);
+        assertTrue(list1.containsAll(arrayList));
+    }
+    
+    @Test
+    public void testAddAll() throws Exception {
+        EncodedColumnQualiferCellsList list1 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list1);
+        EncodedColumnQualiferCellsList list2 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list2);
+        /* 
+         * Note that we don't care about equality of the element being added with the element already
+         * present at the index.
+         */
+        assertTrue(list1.addAll(list2));
+    }
+    
+    @Test
+    public void testAddAllAtIndexFails() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list);
+        try {
+            list.addAll(0, new ArrayList<Cell>());
+        } catch (UnsupportedOperationException expected) {
+        }
+    }
+    
+    @Test
+    public void testRemoveAll() throws Exception {
+        EncodedColumnQualiferCellsList list1 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list1);
+        ArrayList<Cell> list2 = new ArrayList<>();
+        populateList(list2);
+        assertTrue(list1.removeAll(list2));
+        assertTrue(list1.isEmpty());
+        assertFalse(list2.isEmpty());
+    }
+    
+    @Test
+    public void testRetainAll() throws Exception {
+        EncodedColumnQualiferCellsList list1 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list1);
+        EncodedColumnQualiferCellsList list2 = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list2);
+        // retainAll won't be modifying the list1 since they both have the same elements equality wise
+        assertFalse(list1.retainAll(list2));
+        list2.remove(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(12)));
+        assertTrue(list1.retainAll(list2));
+        assertEquals(list1.size(), list2.size());
+        for (Cell c : list1) {
+            assertTrue(list2.contains(c));
+        }
+    }
+    
+    @Test
+    public void testClear() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list);
+        list.clear();
+        assertTrue(list.isEmpty());
+        assertEquals(0, list.size());
+    }
+    
+    @Test
+    public void testGetIndex() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        for (int i = 0; i < cells.length; i++) {
+            assertEquals(cells[i], list.get(i));
+        }
+    }
+    
+    @Test
+    public void testIndexOf() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        for (int i = 0; i < cells.length; i++) {
+            assertEquals(i, list.indexOf(cells[i]));
+        }
+    }
+    
+    @Test
+    public void testLastIndexOf() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        populateListAndArray(list, cells);
+        for (int i = 0; i < cells.length; i++) {
+            assertEquals(i, list.lastIndexOf(cells[i]));
+        }
+    }
+    
+    @Test
+    public void testListIterator() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] cells = new Cell[7];
+        int i = 0;
+        populateListAndArray(list, cells);
+        ListIterator<Cell> itr = list.listIterator();
+        assertTrue(itr.hasNext());
+        
+        // test itr.next()
+        i = 0;
+        while (itr.hasNext()) {
+            assertEquals(cells[i++], itr.next());
+        }
+        
+        assertEquals(7, list.size());
+        
+        // test itr.remove()
+        itr = list.listIterator();
+        i = 0;
+        int numRemoved = 0;
+        try {
+            itr.remove();
+            fail("Remove not allowed till next() is called");
+        } catch (IllegalStateException expected) {}
+        
+        while (itr.hasNext()) {
+            assertEquals(cells[i++], itr.next());
+            itr.remove();
+            numRemoved++;
+        }
+        assertEquals("Number of elements removed should have been the size of the list", 7, numRemoved);
+        assertTrue(list.isEmpty());
+    }
+    
+    @Test
+    public void testListIteratorSet() {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] array = new Cell[7];
+        populateListAndArray(list, array);
+        ListIterator<Cell> itr = list.listIterator();
+        // This cell is KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(12))
+        final Cell validCell = array[4];
+        // This cell is KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(14))
+        final Cell invalidCell = array[5];
+        String validCellName = "Valid Cell";
+        String invalidCellName = "Invalid Cell";
+        Cell validReplacementCell = new DelegateCell(validCell, validCellName);
+        Cell invalidReplacementCell = new DelegateCell(invalidCell, invalidCellName);
+        int i = 0;
+        while (itr.hasNext()) {
+            Cell c = itr.next();
+            if (i == 4) {
+                itr.set(validReplacementCell);
+            }
+            if (i == 6) {
+                try {
+                    itr.set(invalidReplacementCell);
+                    fail("This should have failed since " + invalidReplacementCell + " cannot be added where " + c + " is.");
+                } catch (IllegalArgumentException expected) {
+                }
+            }
+            i++;
+        }
+        itr = list.listIterator();
+        i = 0;
+        // Assert that the valid cell was added and invalid cell wasn't.
+        while (itr.hasNext()) {
+            Cell c = itr.next();
+            if (i == 4) {
+                assertEquals(validCellName, c.toString());
+            }
+            if (i == 6) {
+                assertNotEquals(invalidCellName, c.toString());
+            }
+            i++;
+        }
+    }
+    
+    @Test
+    public void testListIteratorNextAndPrevious()  throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        Cell[] array = new Cell[7];
+        populateListAndArray(list, array);
+        ListIterator<Cell> itr = list.listIterator();
+        try {
+            itr.previous();
+            fail("Call to itr.previous() should have failed since the iterator hasn't been moved forward yet");
+        } catch (NoSuchElementException expected) {
+            
+        }
+        Cell c = itr.next();
+        Cell d = itr.previous();
+        Cell e = itr.next();
+        Cell f = itr.previous();
+        assertTrue(c.equals(d) && c.equals(f) && c.equals(e));
+        itr = list.listIterator();
+        int i = 0;
+        assertEquals(array[i++], itr.next());
+        assertEquals(array[i++], itr.next()); 
+        assertEquals(array[i++], itr.next());
+        assertEquals(array[--i], itr.previous());
+        assertEquals(array[--i], itr.previous());
+        assertEquals(array[i++], itr.next());
+        
+        // move itr forward till next() is exhausted
+        while (itr.hasNext()) {
+            itr.next();
+        }
+        i = 6;
+        while (itr.hasPrevious()) {
+            assertEquals(array[i--], itr.previous());
+        }
+        assertEquals("Not all elements navigated using previous()", -1, i);
+        // now that previous is exhausted, move itr() forward till next() is exhausted
+        i = 0;
+        while (itr.hasNext()) {
+            assertEquals(array[i++], itr.next());
+        }
+        assertEquals("Not all elements navigated using next()", 7, i);
+    }
+    
+    @Test
+    public void testSetNull() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        try {
+            list.add(null);
+            fail("Adding null elements to the list is not allowed");
+        } catch (NullPointerException expected) {
+            
+        }
+    }
+    
+    @Test
+    public void testFailFastIterator() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list);
+        int i = 0;
+        Iterator<Cell> itr = list.iterator();
+        while (itr.hasNext()) {
+            i++;
+            try {
+                itr.next();
+                list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(0)));
+                if (i == 2) {
+                    fail("ConcurrentModificationException should have been thrown as the list is being modified while being iterated through");
+                }
+            } catch (ConcurrentModificationException expected) {
+                assertEquals("Exception should have been thrown when getting the second element",
+                    2, i);
+                break;
+            }
+        }
+    }
+    
+    @Test
+    public void testFailFastListIterator() throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list);
+        ListIterator<Cell> itr = list.listIterator();
+        itr.next();
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(0)));
+        try {
+            itr.next();
+            fail("ConcurrentModificationException should have been thrown as the list was modified without using iterator");
+        } catch (ConcurrentModificationException expected) {
+
+        }
+        list = new EncodedColumnQualiferCellsList(11, 16);
+        populateList(list);
+        itr = list.listIterator();
+        itr.next();
+        itr.next();
+        itr.remove();
+        itr.next();
+        list.remove(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(0)));
+        try {
+            itr.next();
+            fail("ConcurrentModificationException should have been thrown as the list was modified without using iterator");
+        } catch (ConcurrentModificationException expected) {
+
+        }
+    }
+    
+    private void populateListAndArray(List<Cell> list, Cell[] cells) {
+        // add elements in reserved range
+        list.add(cells[0] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(0)));
+        list.add(cells[1] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(5)));
+        list.add(cells[2] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(10)));
+
+        // add elements in qualifier range
+        list.add(cells[6] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(16)));
+        list.add(cells[4] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(12)));
+        list.add(cells[5] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(14)));
+        list.add(cells[3] = KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(11)));
+    }
+
+    private void populateList(List<Cell> list) {
+        // add elements in reserved range
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(0)));
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(5)));
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(10)));
+
+        // add elements in qualifier range
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(16)));
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(12)));
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(14)));
+        list.add(KeyValue.createFirstOnRow(row, cf, getEncodedColumnQualifier(11)));
+    }
+    
+    private class DelegateCell implements Cell {
+        private final Cell delegate;
+        private final String name;
+        public DelegateCell(Cell delegate, String name) {
+            this.delegate = delegate;
+            this.name = name;
+        }
+
+        @Override
+        public int getValueOffset() {
+            return delegate.getValueOffset();
+        }
+
+        @Override
+        public int getValueLength() {
+            return delegate.getValueLength();
+        }
+
+        @Override
+        public byte[] getValueArray() {
+            return delegate.getValueArray();
+        }
+
+        @Override
+        public byte[] getValue() {
+            return delegate.getValue();
+        }
+
+        @Override
+        public byte getTypeByte() {
+            return delegate.getTypeByte();
+        }
+
+        @Override
+        public long getTimestamp() {
+            return delegate.getTimestamp();
+        }
+
+        @Override
+        public int getTagsOffset() {
+            return delegate.getTagsOffset();
+        }
+
+        @Override
+        public int getTagsLengthUnsigned() {
+            return delegate.getTagsLengthUnsigned();
+        }
+
+        @Override
+        public short getTagsLength() {
+            return delegate.getTagsLength();
+        }
+
+        @Override
+        public byte[] getTagsArray() {
+            return delegate.getTagsArray();
+        }
+
+        @Override
+        public int getRowOffset() {
+            return delegate.getRowOffset();
+        }
+
+        @Override
+        public short getRowLength() {
+            return delegate.getRowLength();
+        }
+
+        @Override
+        public byte[] getRowArray() {
+            return delegate.getRowArray();
+        }
+
+        @Override
+        public byte[] getRow() {
+            return delegate.getRow();
+        }
+
+        @Override
+        public int getQualifierOffset() {
+            return delegate.getQualifierOffset();
+        }
+
+        @Override
+        public int getQualifierLength() {
+            return delegate.getQualifierLength();
+        }
+
+        @Override
+        public byte[] getQualifierArray() {
+            return delegate.getQualifierArray();
+        }
+
+        @Override
+        public byte[] getQualifier() {
+            return delegate.getQualifier();
+        }
+
+        @Override
+        public long getMvccVersion() {
+            return delegate.getMvccVersion();
+        }
+
+        @Override
+        public int getFamilyOffset() {
+            return delegate.getFamilyOffset();
+        }
+
+        @Override
+        public byte getFamilyLength() {
+            return delegate.getFamilyLength();
+        }
+
+        @Override
+        public byte[] getFamilyArray() {
+            return delegate.getFamilyArray();
+        }
+
+        @Override
+        public byte[] getFamily() {
+            return delegate.getFamily();
+        }
+        
+        @Override
+        public String toString() {
+            return name;
+        }
+
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 5feedb1..5409554 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -304,11 +304,11 @@ public class TestUtil {
     }
 
     public static Expression constantComparison(CompareOp op, PColumn c, Object o) {
-        return  new ComparisonExpression(Arrays.<Expression>asList(new KeyValueColumnExpression(c), LiteralExpression.newConstant(o)), op);
+        return  new ComparisonExpression(Arrays.<Expression>asList(new KeyValueColumnExpression(c, true), LiteralExpression.newConstant(o)), op);
     }
 
     public static Expression kvColumn(PColumn c) {
-        return new KeyValueColumnExpression(c);
+        return new KeyValueColumnExpression(c, true);
     }
 
     public static Expression pkColumn(PColumn c, List<PColumn> columns) {
@@ -610,7 +610,7 @@ public class TestUtil {
     }
 
     public static void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
-    	analyzeTable(conn, tableName, false);
+        analyzeTable(conn, tableName, false);
     }
     
     public static void analyzeTable(Connection conn, String tableName, boolean transactional) throws IOException, SQLException {
@@ -652,17 +652,17 @@ public class TestUtil {
         Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * MILLIS_IN_DAY);
         stmt.setDate(6, date);
     }
-	
+    
     public static void validateRowKeyColumns(ResultSet rs, int i) throws SQLException {
-		assertTrue(rs.next());
-		assertEquals(rs.getString(1), "varchar" + String.valueOf(i));
-		assertEquals(rs.getString(2), "char" + String.valueOf(i));
-		assertEquals(rs.getInt(3), i);
-		assertEquals(rs.getInt(4), i);
-		assertEquals(rs.getBigDecimal(5), new BigDecimal(i*0.5d));
-		Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * MILLIS_IN_DAY);
-		assertEquals(rs.getDate(6), date);
-	}
+        assertTrue(rs.next());
+        assertEquals(rs.getString(1), "varchar" + String.valueOf(i));
+        assertEquals(rs.getString(2), "char" + String.valueOf(i));
+        assertEquals(rs.getInt(3), i);
+        assertEquals(rs.getInt(4), i);
+        assertEquals(rs.getBigDecimal(5), new BigDecimal(i*0.5d));
+        Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * MILLIS_IN_DAY);
+        assertEquals(rs.getDate(6), date);
+    }
     
     public static String getTableName(Boolean mutable, Boolean transactional) {
         StringBuilder tableNameBuilder = new StringBuilder(DEFAULT_DATA_TABLE_NAME);
@@ -694,7 +694,7 @@ public class TestUtil {
                 
                 @Override
                 public SortOrder getSortOrder() {
-                	return SortOrder.getDefault();
+                    return SortOrder.getDefault();
                 }
                 
                 @Override
@@ -720,11 +720,15 @@ public class TestUtil {
                 public boolean isRowTimestamp() {
                     return false;
                 }
-    			@Override
-    			public boolean isDynamic() {
-    				return false;
-    			}
-            })), null);
+                @Override
+                public boolean isDynamic() {
+                    return false;
+                }
+                @Override
+                public Integer getEncodedColumnQualifier() {
+                    return null;
+                }
+            }, false)), null);
             aggregationManager.setAggregators(new ClientAggregators(Collections.<SingleAggregateFunction>singletonList(func), 1));
             ClientAggregators aggregators = aggregationManager.getAggregators();
             return aggregators;
@@ -821,4 +825,3 @@ public class TestUtil {
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index a16263f..d5df2f3 100644
--- a/phoenix-protocol/src/main/PTable.proto
+++ b/phoenix-protocol/src/main/PTable.proto
@@ -47,6 +47,7 @@ message PColumn {
   optional string expression = 12;
   optional bool isRowTimestamp = 13;
   optional bool isDynamic = 14;
+  optional int32 columnQualifier = 15;
 }
 
 message PTableStats {
@@ -95,4 +96,11 @@ message PTable {
   optional string autoParititonSeqName = 31;
   optional bool isAppendOnlySchema = 32;
   optional bytes parentNameBytes = 33;
+  optional bytes storageScheme = 34;
+  repeated EncodedCQCounter encodedCQCounters = 35;
+}
+
+message EncodedCQCounter {
+	required string colFamily = 1;
+	required int32 counter = 2;
 }


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 8e4d9aa..f5df980 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.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
@@ -125,10 +126,12 @@ public class FromCompiler {
             throw new ColumnNotFoundException(schemaName, tableName, null, colName);
         }
         
+        @Override
         public PFunction resolveFunction(String functionName) throws SQLException {
             throw new FunctionNotFoundException(functionName);
         }
 
+        @Override
         public boolean hasUDFs() {
             return false;
         }
@@ -185,7 +188,7 @@ public class FromCompiler {
                     if (htable != null) Closeables.closeQuietly(htable);
                 }
                 tableNode = NamedTableNode.create(null, baseTable, statement.getColumnDefs());
-                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1), false);
+                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1), false, false);
             }
             throw e;
         }
@@ -257,7 +260,7 @@ public class FromCompiler {
             Expression sourceExpression = projector.getColumnProjector(column.getPosition()).getExpression();
             PColumnImpl projectedColumn = new PColumnImpl(column.getName(), column.getFamilyName(),
                     sourceExpression.getDataType(), sourceExpression.getMaxLength(), sourceExpression.getScale(), sourceExpression.isNullable(),
-                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic());
+                    column.getPosition(), sourceExpression.getSortOrder(), column.getArraySize(), column.getViewConstant(), column.isViewReferenced(), column.getExpressionStr(), column.isRowTimestamp(), column.isDynamic(), column.getEncodedColumnQualifier());
             projectedColumns.add(projectedColumn);
         }
         PTable t = PTableImpl.makePTable(table, projectedColumns);
@@ -332,26 +335,26 @@ public class FromCompiler {
     	private final String alias;
         private final List<PSchema> schemas;
 
-       public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes, boolean isNamespaceMapped) throws SQLException  {
-           super(connection, 0, false, udfParseNodes);
-           List<PColumnFamily> families = Lists.newArrayListWithExpectedSize(table.getDynamicColumns().size());
-           for (ColumnDef def : table.getDynamicColumns()) {
-               if (def.getColumnDefName().getFamilyName() != null) {
-                   families.add(new PColumnFamilyImpl(PNameFactory.newName(def.getColumnDefName().getFamilyName()),Collections.<PColumn>emptyList()));
-               }
+        public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode table, long timeStamp, Map<String, UDFParseNode> udfParseNodes, boolean isNamespaceMapped, boolean useEncodedColumnNames) throws SQLException {
+            super(connection, 0, false, udfParseNodes);
+            List<PColumnFamily> families = Lists.newArrayListWithExpectedSize(table.getDynamicColumns().size());
+            for (ColumnDef def : table.getDynamicColumns()) {
+                if (def.getColumnDefName().getFamilyName() != null) {
+                    families.add(new PColumnFamilyImpl(PNameFactory.newName(def.getColumnDefName().getFamilyName()),Collections.<PColumn>emptyList(), useEncodedColumnNames));
+                }
             }
             Long scn = connection.getSCN();
             String schema = table.getName().getSchemaName();
             if (connection.getSchema() != null) {
                 schema = schema != null ? schema : connection.getSchema();
             }
-           PTable theTable = new PTableImpl(connection.getTenantId(), schema, table.getName().getTableName(),
+            PTable theTable = new PTableImpl(connection.getTenantId(), schema, table.getName().getTableName(),
                     scn == null ? HConstants.LATEST_TIMESTAMP : scn, families, isNamespaceMapped);
-           theTable = this.addDynamicColumns(table.getDynamicColumns(), theTable);
-           alias = null;
-           tableRefs = ImmutableList.of(new TableRef(alias, theTable, timeStamp, !table.getDynamicColumns().isEmpty()));
-           schemas = ImmutableList.of(new PSchema(theTable.getSchemaName().toString(), timeStamp));
-       }
+            theTable = this.addDynamicColumns(table.getDynamicColumns(), theTable);
+            alias = null;
+            tableRefs = ImmutableList.of(new TableRef(alias, theTable, timeStamp, !table.getDynamicColumns().isEmpty()));
+            schemas = ImmutableList.of(new PSchema(theTable.getSchemaName().toString(), timeStamp));
+        }
 
         public SingleTableColumnResolver(PhoenixConnection connection, NamedTableNode tableNode, boolean updateCacheImmediately) throws SQLException {
             this(connection, tableNode, updateCacheImmediately, 0, new HashMap<String,UDFParseNode>(1));
@@ -447,8 +450,8 @@ public class FromCompiler {
 
 			}
         	PColumn column = resolveCF
-        	        ? tableRef.getTable().getColumnFamily(tableName).getColumn(colName)
-        			: tableRef.getTable().getColumn(colName);
+        	        ? tableRef.getTable().getColumnFamily(tableName).getPColumnForColumnName(colName)
+        			: tableRef.getTable().getPColumnForColumnName(colName);
             return new ColumnRef(tableRef, column.getPosition());
 		}
 
@@ -671,8 +674,9 @@ public class FromCompiler {
                         theTable.getColumnFamily(family); // Verifies that column family exists
                         familyName = PNameFactory.newName(family);
                     }
+                    // Dynamic columns don't have an encoded column name
                     allcolumns.add(new PColumnImpl(name, familyName, dynColumn.getDataType(), dynColumn.getMaxLength(),
-                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true));
+                            dynColumn.getScale(), dynColumn.isNull(), position, dynColumn.getSortOrder(), dynColumn.getArraySize(), null, false, dynColumn.getExpression(), false, true, null));
                     position++;
                 }
                 theTable = PTableImpl.makePTable(theTable, allcolumns);
@@ -776,14 +780,14 @@ public class FromCompiler {
                 }
                 PColumnImpl column = new PColumnImpl(PNameFactory.newName(alias),
                         PNameFactory.newName(QueryConstants.DEFAULT_COLUMN_FAMILY),
-                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false);
+                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false, null);
                 columns.add(column);
             }
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
                     MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
                     Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
                     false, null, null, null, false, false, 0, 0L, SchemaUtil
-                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false);
+                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);
@@ -858,7 +862,7 @@ public class FromCompiler {
                 while (iterator.hasNext()) {
                     TableRef tableRef = iterator.next();
                     try {
-                        PColumn column = tableRef.getTable().getColumn(colName);
+                        PColumn column = tableRef.getTable().getPColumnForColumnName(colName);
                         if (theTableRef != null) { throw new AmbiguousColumnException(colName); }
                         theTableRef = tableRef;
                         theColumnPosition = column.getPosition();
@@ -871,12 +875,12 @@ public class FromCompiler {
             } else {
                 try {
                     TableRef tableRef = resolveTable(schemaName, tableName);
-                    PColumn column = tableRef.getTable().getColumn(colName);
+                    PColumn column = tableRef.getTable().getPColumnForColumnName(colName);
                     return new ColumnRef(tableRef, column.getPosition());
                 } catch (TableNotFoundException e) {
                     // Try using the tableName as a columnFamily reference instead
                     ColumnFamilyRef cfRef = resolveColumnFamily(schemaName, tableName);
-                    PColumn column = cfRef.getFamily().getColumn(colName);
+                    PColumn column = cfRef.getFamily().getPColumnForColumnName(colName);
                     return new ColumnRef(cfRef.getTableRef(), column.getPosition());
                 }
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 e8c05ca..489b993 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.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ProjectedColumn;
@@ -93,6 +94,7 @@ import org.apache.phoenix.schema.types.PTimestamp;
 import org.apache.phoenix.schema.types.PTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -714,7 +716,7 @@ public class JoinCompiler {
                 if (columnRef.getTableRef().equals(tableRef)
                         && !SchemaUtil.isPKColumn(columnRef.getColumn())
                         && !(columnRef instanceof LocalIndexColumnRef)) {
-                    scan.addColumn(columnRef.getColumn().getFamilyName().getBytes(), columnRef.getColumn().getName().getBytes());
+                	EncodedColumnsUtil.setColumns(columnRef.getColumn(), tableRef.getTable(), scan);
                 }
             }
         }
@@ -1301,7 +1303,7 @@ public class JoinCompiler {
         if (left.getBucketNum() != null) {
             merged.remove(0);
         }
-        
+        //TODO: samarth should projected join table should always have non-encoded column names? Is this where we also decide that once we start supporting joins then have the storage scheme right.
         return PTableImpl.makePTable(left.getTenantId(), left.getSchemaName(),
                 PNameFactory.newName(SchemaUtil.getTableName(left.getName().getString(), right.getName().getString())),
                 left.getType(), left.getIndexState(), left.getTimeStamp(), left.getSequenceNumber(), left.getPKName(),
@@ -1310,7 +1312,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());
+                left.getAutoPartitionSeqName(), left.isAppendOnlySchema(), StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
     }
 
 }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
index 393499a..3820ee5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
@@ -50,6 +50,7 @@ import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.TransactionUtil;
 
@@ -176,8 +177,8 @@ public class PostDDLCompiler {
                             @Override
                             public ColumnRef resolveColumn(String schemaName, String tableName, String colName) throws SQLException {
                                 PColumn column = tableName != null
-                                        ? tableRef.getTable().getColumnFamily(tableName).getColumn(colName)
-                                        : tableRef.getTable().getColumn(colName);
+                                        ? tableRef.getTable().getColumnFamily(tableName).getPColumnForColumnName(colName)
+                                        : tableRef.getTable().getPColumnForColumnName(colName);
                                 return new ColumnRef(tableRef, column.getPosition());
                             }
                             
@@ -213,6 +214,7 @@ public class PostDDLCompiler {
                         ScanUtil.setTimeRange(scan, scan.getTimeRange().getMin(), ts);
                         if (emptyCF != null) {
                             scan.setAttribute(BaseScannerRegionObserver.EMPTY_CF, emptyCF);
+                            scan.setAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER, EncodedColumnsUtil.getEmptyKeyValueInfo(tableRef.getTable()).getFirst());
                         }
                         ServerCache cache = null;
                         try {
@@ -236,11 +238,12 @@ public class PostDDLCompiler {
                                     // data empty column family to stay the same, while the index empty column family
                                     // changes.
                                     PColumn column = deleteList.get(0);
+                                    byte[] cq = EncodedColumnsUtil.getColumnQualifier(column, tableRef.getTable());
                                     if (emptyCF == null) {
-                                        scan.addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+                                        scan.addColumn(column.getFamilyName().getBytes(), cq);
                                     }
                                     scan.setAttribute(BaseScannerRegionObserver.DELETE_CF, column.getFamilyName().getBytes());
-                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_CQ, column.getName().getBytes());
+                                    scan.setAttribute(BaseScannerRegionObserver.DELETE_CQ, cq);
                                 }
                             }
                             List<byte[]> columnFamilies = Lists.newArrayListWithExpectedSize(tableRef.getTable().getColumnFamilies().size());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
index 81dbe0d..d6fd6ff 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
@@ -31,6 +31,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -82,7 +83,11 @@ public class PostLocalIndexDDLCompiler {
             // However, in this case, we need to project all of the data columns that contribute to the index.
             IndexMaintainer indexMaintainer = index.getIndexMaintainer(dataTable, connection);
             for (ColumnReference columnRef : indexMaintainer.getAllColumns()) {
-                scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
+                if (index.getStorageScheme() == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL) {
+                    scan.addFamily(columnRef.getFamily());
+                } else {
+                    scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
+                }
             }
 
             // Go through MutationPlan abstraction so that we can create local indexes

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
index 99a9731..25aee35 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/ProjectionCompiler.java
@@ -88,6 +88,7 @@ import org.apache.phoenix.schema.ValueBitSet;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
@@ -217,7 +218,7 @@ public class ProjectionCompiler {
             PColumn indexColumn = null;
             ColumnRef ref = null;
             try {
-                indexColumn = index.getColumn(indexColName);
+                indexColumn = index.getPColumnForColumnName(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
             } catch (ColumnNotFoundException e) {
                 if (index.getIndexType() == IndexType.LOCAL) {
@@ -289,7 +290,7 @@ public class ProjectionCompiler {
             ColumnRef ref = null;
             String indexColumnFamily = null;
             try {
-                indexColumn = index.getColumn(indexColName);
+                indexColumn = index.getPColumnForColumnName(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
                 indexColumnFamily = indexColumn.getFamilyName() == null ? null : indexColumn.getFamilyName().getString();
             } catch (ColumnNotFoundException e) {
@@ -484,7 +485,7 @@ public class ProjectionCompiler {
                 }
             } else {
                 for (byte[] cq : entry.getValue()) {
-                    PColumn column = family.getColumn(cq);
+                    PColumn column = family.getPColumnForColumnQualifier(cq);
                     Integer maxLength = column.getMaxLength();
                     int byteSize = column.getDataType().isFixedWidth() ? maxLength == null ? column.getDataType().getByteSize() : maxLength : RowKeySchema.ESTIMATED_VARIABLE_LENGTH_SIZE;
                     estimatedByteSize += SizedUtil.KEY_VALUE_SIZE + estimatedKeySize + byteSize;
@@ -663,7 +664,8 @@ public class ProjectionCompiler {
                      public Void visit(ProjectedColumnExpression expression) {
                          if (expression.getDataType().isArrayType()) {
                              indexProjectedColumns.add(expression);
-                             KeyValueColumnExpression keyValueColumnExpression = new KeyValueColumnExpression(expression.getColumn());
+                             PColumn col = expression.getColumn();
+                             KeyValueColumnExpression keyValueColumnExpression = new KeyValueColumnExpression(col, EncodedColumnsUtil.hasEncodedColumnName(col));
                              indexKVs.add(keyValueColumnExpression);
                              copyOfChildren.set(0, keyValueColumnExpression);
                              Integer count = arrayExpressionCounts.get(expression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
index 2258f28..5126c8b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/QueryCompiler.java
@@ -94,7 +94,7 @@ public class QueryCompiler {
      */
     private static final String LOAD_COLUMN_FAMILIES_ON_DEMAND_ATTR = "_ondemand_";
     private final PhoenixStatement statement;
-    private final Scan scan;
+	private final Scan scan;
     private final Scan originalScan;
     private final ColumnResolver resolver;
     private final SelectStatement select;

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 4d3c0cf..c472039 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
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -24,7 +26,6 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.ColumnParseNode;
 import org.apache.phoenix.parse.FamilyWildcardParseNode;
@@ -43,11 +44,14 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.EncodedCQCounter;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ProjectedColumn;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -136,7 +140,7 @@ public class TupleProjectionCompiler {
             projectedColumns.add(column);
             // Wildcard or FamilyWildcard will be handled by ProjectionCompiler.
             if (!isWildcard && !families.contains(sourceColumn.getFamilyName())) {
-                context.getScan().addColumn(sourceColumn.getFamilyName().getBytes(), sourceColumn.getName().getBytes());
+            	EncodedColumnsUtil.setColumns(column, table, context.getScan());
             }
         }
         // add LocalIndexDataColumnRef
@@ -154,9 +158,9 @@ 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.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getStorageScheme(), table.getEncodedCQCounter());
     }
-
+    
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
         PTable table = tableRef.getTable();
         boolean hasSaltingColumn = retainPKColumns && table.getBucketNum() != null;
@@ -172,17 +176,22 @@ public class TupleProjectionCompiler {
 
             PColumn column = new ProjectedColumn(PNameFactory.newName(aliasedName), 
                     retainPKColumns && SchemaUtil.isPKColumn(sourceColumn) ? 
-                            null : PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), 
+                            null : PNameFactory.newName(VALUE_COLUMN_FAMILY), 
                     position++, sourceColumn.isNullable(), sourceColumnRef);
             projectedColumns.add(column);
         }
+        StorageScheme storageScheme = table.getStorageScheme();
+        EncodedCQCounter cqCounter = EncodedCQCounter.NULL_COUNTER;
+        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());
+                table.getUpdateCacheFrequency(), table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), storageScheme, cqCounter);
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 bc3466c..d779df1 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.StorageScheme;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -88,7 +89,7 @@ public class UnionCompiler {
                 UNION_FAMILY_NAME, targetTypes.get(i).getType(), targetTypes.get(i).getMaxLength(),
                 targetTypes.get(i).getScale(), colProj.getExpression().isNullable(), i,
                 targetTypes.get(i).getSortOrder(), 500, null, false,
-                colProj.getExpression().toString(), false, false);
+                colProj.getExpression().toString(), false, false, null);
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();
@@ -98,7 +99,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);
+                statement.getConnection().getQueryServices().getProps()), null, false, StorageScheme.NON_ENCODED_COLUMN_NAMES, PTable.EncodedCQCounter.NULL_COUNTER);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);
         return tableRef;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 8837445..fbe07df 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -903,10 +903,11 @@ public class UpsertCompiler {
                 UpdateColumnCompiler compiler = new UpdateColumnCompiler(context);
                 int nColumns = onDupKeyPairs.size();
                 List<Expression> updateExpressions = Lists.newArrayListWithExpectedSize(nColumns);
-                LinkedHashSet<PColumn>updateColumns = Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
+                LinkedHashSet<PColumn> updateColumns = Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
+                //TODO: samarth verify if we should pass the column qualifier as null here.
                 updateColumns.add(new PColumnImpl(
                         table.getPKColumns().get(0).getName(), // Use first PK column name as we know it won't conflict with others
-                        null, PVarbinary.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false));
+                        null, PVarbinary.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null));
                 for (Pair<ColumnName,ParseNode> columnPair : onDupKeyPairs) {
                     ColumnName colName = columnPair.getFirst();
                     PColumn updateColumn = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
index 13963d7..59c287e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/WhereCompiler.java
@@ -51,11 +51,13 @@ import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.ColumnRef;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.TypeMismatchException;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -168,12 +170,14 @@ public class WhereCompiler {
         public Expression visit(ColumnParseNode node) throws SQLException {
             ColumnRef ref = resolveColumn(node);
             TableRef tableRef = ref.getTableRef();
+            Expression newColumnExpression = ref.newColumnExpression(node.isTableNameCaseSensitive(), node.isCaseSensitive());
             if (tableRef.equals(context.getCurrentTable()) && !SchemaUtil.isPKColumn(ref.getColumn())) {
+                byte[] cq = tableRef.getTable().getStorageScheme() == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL 
+                		? ref.getColumn().getFamilyName().getBytes() : EncodedColumnsUtil.getColumnQualifier(ref.getColumn(), tableRef.getTable());
                 // track the where condition columns. Later we need to ensure the Scan in HRS scans these column CFs
-                context.addWhereCoditionColumn(ref.getColumn().getFamilyName().getBytes(), ref.getColumn().getName()
-                        .getBytes());
+                context.addWhereCoditionColumn(ref.getColumn().getFamilyName().getBytes(), cq);
             }
-            return ref.newColumnExpression(node.isTableNameCaseSensitive(), node.isCaseSensitive());
+			return newColumnExpression;
         }
 
         @Override
@@ -194,7 +198,7 @@ public class WhereCompiler {
             // just use that.
             try {
                 if (!SchemaUtil.isPKColumn(ref.getColumn())) {
-                    table.getColumn(ref.getColumn().getName().getString());
+                    table.getPColumnForColumnName(ref.getColumn().getName().getString());
                 }
             } catch (AmbiguousColumnException e) {
                 disambiguateWithFamily = true;
@@ -222,6 +226,7 @@ public class WhereCompiler {
 
             }
         }
+        
         public Count getCount() {
             return count;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
index ba9f7c8..9589c37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseScannerRegionObserver.java
@@ -50,6 +50,8 @@ import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.ValueBitSet;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.PositionBasedMultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.PositionBasedResultTuple;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.IndexUtil;
@@ -77,12 +79,14 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     public static final String DELETE_CQ = "_DeleteCQ";
     public static final String DELETE_CF = "_DeleteCF";
     public static final String EMPTY_CF = "_EmptyCF";
+    public static final String EMPTY_COLUMN_QUALIFIER = "_EmptyColumnQualifier";
     public static final String SPECIFIC_ARRAY_INDEX = "_SpecificArrayIndex";
     public static final String GROUP_BY_LIMIT = "_GroupByLimit";
     public static final String LOCAL_INDEX = "_LocalIndex";
     public static final String LOCAL_INDEX_BUILD = "_LocalIndexBuild";
     public static final String LOCAL_INDEX_JOIN_SCHEMA = "_LocalIndexJoinSchema";
     public static final String DATA_TABLE_COLUMNS_TO_JOIN = "_DataTableColumnsToJoin";
+    public static final String COLUMNS_STORED_IN_SINGLE_CELL = "_ColumnsStoredInSingleCell";
     public static final String VIEW_CONSTANTS = "_ViewConstants";
     public static final String EXPECTED_UPPER_REGION_KEY = "_ExpectedUpperRegionKey";
     public static final String REVERSE_SCAN = "_ReverseScan";
@@ -101,6 +105,8 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     public final static String SCAN_OFFSET = "_RowOffset";
     public static final String SCAN_START_ROW_SUFFIX = "_ScanStartRowSuffix";
     public static final String SCAN_STOP_ROW_SUFFIX = "_ScanStopRowSuffix";
+    public final static String MIN_QUALIFIER = "_MinQualifier";
+    public final static String MAX_QUALIFIER = "_MaxQualifier";
     
     /**
      * Attribute name used to pass custom annotations in Scans and Mutations (later). Custom annotations
@@ -307,14 +313,14 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
      * @param indexMaintainer
      * @param viewConstants
      */
-    protected RegionScanner getWrappedScanner(final ObserverContext<RegionCoprocessorEnvironment> c,
+    RegionScanner getWrappedScanner(final ObserverContext<RegionCoprocessorEnvironment> c,
             final RegionScanner s, final int offset, final Scan scan,
             final ColumnReference[] dataColumns, final TupleProjector tupleProjector,
             final HRegion dataRegion, final IndexMaintainer indexMaintainer,
             final byte[][] viewConstants, final TupleProjector projector,
-            final ImmutableBytesWritable ptr) {
+            final ImmutableBytesWritable ptr, final boolean useQualiferAsListIndex) {
         return getWrappedScanner(c, s, null, null, offset, scan, dataColumns, tupleProjector,
-                dataRegion, indexMaintainer, null, viewConstants, null, null, projector, ptr);
+                dataRegion, indexMaintainer, null, viewConstants, null, null, projector, ptr, useQualiferAsListIndex);
     }
     
     /**
@@ -332,7 +338,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
      * @param tx current transaction
      * @param viewConstants
      */
-    protected RegionScanner getWrappedScanner(final ObserverContext<RegionCoprocessorEnvironment> c,
+    RegionScanner getWrappedScanner(final ObserverContext<RegionCoprocessorEnvironment> c,
             final RegionScanner s, final Set<KeyValueColumnExpression> arrayKVRefs,
             final Expression[] arrayFuncRefs, final int offset, final Scan scan,
             final ColumnReference[] dataColumns, final TupleProjector tupleProjector,
@@ -340,7 +346,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             Transaction tx, 
             final byte[][] viewConstants, final KeyValueSchema kvSchema, 
             final ValueBitSet kvSchemaBitSet, final TupleProjector projector,
-            final ImmutableBytesWritable ptr) {
+            final ImmutableBytesWritable ptr, final boolean useQualifierAsListIndex) {
         return new RegionScanner() {
 
             private boolean hasReferences = checkForReferenceFiles();
@@ -437,11 +443,13 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                             tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
                     }
                     if (projector != null) {
-                        Tuple tuple = projector.projectResults(new ResultTuple(Result.create(result)));
+                        Tuple toProject = useQualifierAsListIndex ? new PositionBasedResultTuple(result) : new ResultTuple(Result.create(result));
+                        Tuple tuple = projector.projectResults(toProject);
                         result.clear();
                         result.add(tuple.getValue(0));
-                        if(arrayElementCell != null)
+                        if (arrayElementCell != null) {
                             result.add(arrayElementCell);
+                        }
                     }
                     // There is a scanattribute set to retrieve the specific array element
                     return next;
@@ -475,7 +483,8 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                             tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
                     }
                     if (projector != null) {
-                        Tuple tuple = projector.projectResults(new ResultTuple(Result.create(result)));
+                        Tuple toProject = useQualifierAsListIndex ? new PositionBasedMultiKeyValueTuple(result) : new ResultTuple(Result.create(result));
+                        Tuple tuple = projector.projectResults(toProject);
                         result.clear();
                         result.add(tuple.getValue(0));
                         if(arrayElementCell != null)
@@ -528,21 +537,6 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 // Using KeyValueSchema to set and retrieve the value
                 // collect the first kv to get the row
                 Cell rowKv = result.get(0);
-                for (KeyValueColumnExpression kvExp : arrayKVRefs) {
-                    if (kvExp.evaluate(tuple, ptr)) {
-                        for (int idx = tuple.size() - 1; idx >= 0; idx--) {
-                            Cell kv = tuple.getValue(idx);
-                            if (Bytes.equals(kvExp.getColumnFamily(), 0, kvExp.getColumnFamily().length,
-                                    kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength())
-                                && Bytes.equals(kvExp.getColumnName(), 0, kvExp.getColumnName().length,
-                                        kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())) {
-                                // remove the kv that has the full array values.
-                                result.remove(idx);
-                                break;
-                            }
-                        }
-                    }
-                }
                 byte[] value = kvSchema.toBytes(tuple, arrayFuncRefs,
                         kvSchemaBitSet, ptr);
                 // Add a dummy kv with the exact value of the array index

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
index 8cb6dac..0843ba2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DelegateRegionScanner.java
@@ -56,22 +56,27 @@ public class DelegateRegionScanner implements RegionScanner {
         delegate.close();
     }
 
+    @Override
     public long getMaxResultSize() {
         return delegate.getMaxResultSize();
     }
 
+    @Override
     public boolean next(List<Cell> arg0, int arg1) throws IOException {
         return delegate.next(arg0, arg1);
     }
 
+    @Override
     public boolean next(List<Cell> arg0) throws IOException {
         return delegate.next(arg0);
     }
 
+    @Override
     public boolean nextRaw(List<Cell> arg0, int arg1) throws IOException {
         return delegate.nextRaw(arg0, arg1);
     }
 
+    @Override
     public boolean nextRaw(List<Cell> arg0) throws IOException {
         return delegate.nextRaw(arg0);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
index 49e3d71..66cd288 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/GroupedAggregateRegionObserver.java
@@ -24,6 +24,7 @@ import static org.apache.phoenix.query.QueryServices.GROUPBY_ESTIMATED_DISTINCT_
 import static org.apache.phoenix.query.QueryServices.GROUPBY_SPILLABLE_ATTRIB;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_GROUPBY_ESTIMATED_DISTINCT_VALUES;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_GROUPBY_SPILLABLE;
+import static org.apache.phoenix.util.ScanUtil.getMinMaxQualifiersFromScan;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -46,6 +47,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.TenantCache;
@@ -62,7 +64,10 @@ import org.apache.phoenix.join.HashJoinInfo;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.tuple.EncodedColumnQualiferCellsList;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.PositionBasedMultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.Closeables;
 import org.apache.phoenix.util.IndexUtil;
@@ -131,6 +136,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
 
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        boolean useQualifierAsIndex = ScanUtil.useQualifierAsIndex(ScanUtil.getMinMaxQualifiersFromScan(scan));
         if (ScanUtil.isLocalIndex(scan) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
@@ -139,13 +145,13 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             ImmutableBytesPtr tempPtr = new ImmutableBytesPtr();
             innerScanner =
                     getWrappedScanner(c, innerScanner, offset, scan, dataColumns, tupleProjector, 
-                            c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
+                            c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr, useQualifierAsIndex);
         } 
 
         if (j != null) {
             innerScanner =
                     new HashJoinRegionScanner(innerScanner, p, j, ScanUtil.getTenantId(scan),
-                            c.getEnvironment());
+                            c.getEnvironment(), useQualifierAsIndex);
         }
 
         long limit = Long.MAX_VALUE;
@@ -377,7 +383,8 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             estDistVals = Math.max(MIN_DISTINCT_VALUES, 
                             (int) (Bytes.toInt(estDistValsBytes) * 1.5f));
         }
-
+        Pair<Integer, Integer> minMaxQualifiers = getMinMaxQualifiersFromScan(scan);
+        boolean useQualifierAsIndex = ScanUtil.useQualifierAsIndex(ScanUtil.getMinMaxQualifiersFromScan(scan));
         final boolean spillableEnabled =
                 conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
 
@@ -388,12 +395,10 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         boolean success = false;
         try {
             boolean hasMore;
-
-            MultiKeyValueTuple result = new MultiKeyValueTuple();
+            Tuple result = useQualifierAsIndex ? new PositionBasedMultiKeyValueTuple() : new MultiKeyValueTuple();
             if (logger.isDebugEnabled()) {
                 logger.debug(LogUtil.addCustomAnnotations("Spillable groupby enabled: " + spillableEnabled, ScanUtil.getCustomAnnotations(scan)));
             }
-
             HRegion region = c.getEnvironment().getRegion();
             boolean acquiredLock = false;
             try {
@@ -401,7 +406,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 acquiredLock = true;
                 synchronized (scanner) {
                     do {
-                        List<Cell> results = new ArrayList<Cell>();
+                        List<Cell> results = useQualifierAsIndex ? new EncodedColumnQualiferCellsList(minMaxQualifiers.getFirst(), minMaxQualifiers.getSecond()) : new ArrayList<Cell>();
                         // Results are potentially returned even when the return
                         // value of s.next is false
                         // since this is an indication of whether or not there are
@@ -436,7 +441,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             }
         }
     }
-
+    
     /**
      * Used for an aggregate query in which the key order match the group by key order. In this
      * case, we can do the aggregation as we scan, by detecting when the group by key changes.
@@ -451,6 +456,8 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             logger.debug(LogUtil.addCustomAnnotations("Grouped aggregation over ordered rows with scan " + scan + ", group by "
                     + expressions + ", aggregators " + aggregators, ScanUtil.getCustomAnnotations(scan)));
         }
+        final Pair<Integer, Integer> minMaxQualifiers = getMinMaxQualifiersFromScan(scan);
+        final boolean useQualifierAsIndex = ScanUtil.useQualifierAsIndex(minMaxQualifiers);
         return new BaseRegionScanner(scanner) {
             private long rowCount = 0;
             private ImmutableBytesPtr currentKey = null;
@@ -460,7 +467,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                 boolean hasMore;
                 boolean atLimit;
                 boolean aggBoundary = false;
-                MultiKeyValueTuple result = new MultiKeyValueTuple();
+                Tuple result = useQualifierAsIndex ? new PositionBasedMultiKeyValueTuple() : new MultiKeyValueTuple();
                 ImmutableBytesPtr key = null;
                 Aggregator[] rowAggregators = aggregators.getAggregators();
                 // If we're calculating no aggregate functions, we can exit at the
@@ -473,7 +480,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                     acquiredLock = true;
                     synchronized (scanner) {
                         do {
-                            List<Cell> kvs = new ArrayList<Cell>();
+                            List<Cell> kvs = useQualifierAsIndex ? new EncodedColumnQualiferCellsList(minMaxQualifiers.getFirst(), minMaxQualifiers.getSecond()) : new ArrayList<Cell>();
                             // Results are potentially returned even when the return
                             // value of s.next is false
                             // since this is an indication of whether or not there

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
index 480ee6d..1ea6b12 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/HashJoinRegionScanner.java
@@ -42,6 +42,7 @@ import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.tuple.PositionBasedResultTuple;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ServerUtil;
@@ -61,9 +62,10 @@ public class HashJoinRegionScanner implements RegionScanner {
     private List<Tuple>[] tempTuples;
     private ValueBitSet tempDestBitSet;
     private ValueBitSet[] tempSrcBitSet;
+    private final boolean useQualifierAsListIndex;
     
     @SuppressWarnings("unchecked")
-    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesPtr tenantId, RegionCoprocessorEnvironment env) throws IOException {
+    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesPtr tenantId, RegionCoprocessorEnvironment env, boolean useQualifierAsIndex) throws IOException {
         this.env = env;
         this.scanner = scanner;
         this.projector = projector;
@@ -104,13 +106,13 @@ public class HashJoinRegionScanner implements RegionScanner {
             this.tempDestBitSet = ValueBitSet.newInstance(joinInfo.getJoinedSchema());
             this.projector.setValueBitSet(tempDestBitSet);
         }
+        this.useQualifierAsListIndex = useQualifierAsIndex;
     }
     
     private void processResults(List<Cell> result, boolean hasBatchLimit) throws IOException {
         if (result.isEmpty())
             return;
-        
-        Tuple tuple = new ResultTuple(Result.create(result));
+        Tuple tuple = useQualifierAsListIndex ? new PositionBasedResultTuple(result) : new ResultTuple(Result.create(result));
         // For backward compatibility. In new versions, HashJoinInfo.forceProjection()
         // always returns true.
         if (joinInfo.forceProjection()) {
@@ -314,7 +316,6 @@ public class HashJoinRegionScanner implements RegionScanner {
                 processResults(result, limit >= 0);
                 result.clear();
             }
-            
             return nextInQueue(result);
         } catch (Throwable t) {
             ServerUtil.throwIOException(env.getRegion().getRegionNameAsString(), t);


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 9a7b9e3..93a87ea 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
@@ -27,6 +27,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE_BYTES;
@@ -34,6 +35,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYT
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAMILY_NAME_BYTES;
 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.FAMILY_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES;
@@ -57,6 +59,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORAGE_SCHEME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_INDEX;
@@ -74,11 +77,11 @@ import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_
 import static org.apache.phoenix.query.QueryConstants.SEPARATOR_BYTE_ARRAY;
 import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
 import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
 import java.io.IOException;
-import java.sql.DriverManager;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -150,14 +153,12 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
-import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisitor;
-import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
@@ -190,8 +191,10 @@ import org.apache.phoenix.schema.PMetaDataEntity;
 import org.apache.phoenix.schema.PName;
 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.LinkType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
@@ -209,10 +212,12 @@ import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.schema.types.PSmallint;
+import org.apache.phoenix.schema.types.PTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.KeyValueUtil;
@@ -282,6 +287,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             TABLE_FAMILY_BYTES, IS_NAMESPACE_MAPPED_BYTES);
     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 List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
             EMPTY_KEYVALUE_KV,
@@ -308,7 +314,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             UPDATE_CACHE_FREQUENCY_KV,
             IS_NAMESPACE_MAPPED_KV,
             AUTO_PARTITION_SEQ_KV,
-            APPEND_ONLY_SCHEMA_KV
+            APPEND_ONLY_SCHEMA_KV,
+            STORAGE_SCHEME_KV
             );
     static {
         Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
@@ -338,6 +345,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int IS_NAMESPACE_MAPPED_INDEX = TABLE_KV_COLUMNS.indexOf(IS_NAMESPACE_MAPPED_KV);
     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);
 
     // KeyValues for Column
     private static final KeyValue DECIMAL_DIGITS_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
@@ -351,6 +359,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue IS_VIEW_REFERENCED_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_VIEW_REFERENCED_BYTES);
     private static final KeyValue COLUMN_DEF_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_DEF_BYTES);
     private static final KeyValue IS_ROW_TIMESTAMP_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_ROW_TIMESTAMP_BYTES);
+    private static final KeyValue ENCODED_COLUMN_QUALIFIER_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ENCODED_COLUMN_QUALIFIER_BYTES);
     private static final List<KeyValue> COLUMN_KV_COLUMNS = Arrays.<KeyValue>asList(
             DECIMAL_DIGITS_KV,
             COLUMN_SIZE_KV,
@@ -363,11 +372,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             VIEW_CONSTANT_KV,
             IS_VIEW_REFERENCED_KV,
             COLUMN_DEF_KV,
-            IS_ROW_TIMESTAMP_KV
+            IS_ROW_TIMESTAMP_KV,
+            ENCODED_COLUMN_QUALIFIER_KV
             );
     static {
         Collections.sort(COLUMN_KV_COLUMNS, KeyValue.COMPARATOR);
     }
+    private static final KeyValue QUALIFIER_COUNTER_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_QUALIFIER_COUNTER_BYTES);
     private static final int DECIMAL_DIGITS_INDEX = COLUMN_KV_COLUMNS.indexOf(DECIMAL_DIGITS_KV);
     private static final int COLUMN_SIZE_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_SIZE_KV);
     private static final int NULLABLE_INDEX = COLUMN_KV_COLUMNS.indexOf(NULLABLE_KV);
@@ -379,9 +390,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int IS_VIEW_REFERENCED_INDEX = COLUMN_KV_COLUMNS.indexOf(IS_VIEW_REFERENCED_KV);
     private static final int COLUMN_DEF_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_DEF_KV);
     private static final int IS_ROW_TIMESTAMP_INDEX = COLUMN_KV_COLUMNS.indexOf(IS_ROW_TIMESTAMP_KV);
+    private static final int ENCODED_COLUMN_QUALIFIER_INDEX = COLUMN_KV_COLUMNS.indexOf(ENCODED_COLUMN_QUALIFIER_KV);
     
     private static final int LINK_TYPE_INDEX = 0;
-
+    
     private static final KeyValue CLASS_NAME_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, CLASS_NAME_BYTES);
     private static final KeyValue JAR_PATH_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, JAR_PATH_BYTES);
     private static final KeyValue RETURN_TYPE_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, RETURN_TYPE_BYTES);
@@ -717,8 +729,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 isRowTimestampKV == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
                         isRowTimestampKV.getValueArray(), isRowTimestampKV.getValueOffset(),
                         isRowTimestampKV.getValueLength()));
-
-        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false);
+        Cell columnQualifierKV = colKeyValues[ENCODED_COLUMN_QUALIFIER_INDEX];
+        Integer columnQualifier =
+                columnQualifierKV == null ? null : getEncodedColumnQualifier(
+                    columnQualifierKV.getValueArray(), columnQualifierKV.getValueOffset(),
+                    columnQualifierKV.getValueLength());
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false, columnQualifier);
         columns.add(column);
     }
     
@@ -926,37 +942,49 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         boolean isAppendOnlySchema = isAppendOnlySchemaKv == null ? false
                 : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(isAppendOnlySchemaKv.getValueArray(),
                     isAppendOnlySchemaKv.getValueOffset(), isAppendOnlySchemaKv.getValueLength()));
-        
-        
+        Cell storageSchemeKv = tableKeyValues[STORAGE_SCHEME_INDEX];
+        //TODO: change this once we start having other values for storage schemes
+        StorageScheme storageScheme = storageSchemeKv == null ? StorageScheme.NON_ENCODED_COLUMN_NAMES : StorageScheme
+                .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
+                        storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength()));
+
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();
         List<PName> physicalTables = Lists.newArrayList();
         PName parentTableName = tableType == INDEX ? dataTableName : null;
         PName parentSchemaName = tableType == INDEX ? schemaName : null;
+        EncodedCQCounter cqCounter = (storageScheme == StorageScheme.NON_ENCODED_COLUMN_NAMES || tableType == PTableType.VIEW) ? PTable.EncodedCQCounter.NULL_COUNTER : new EncodedCQCounter();
         while (true) {
-          results.clear();
-          scanner.next(results);
-          if (results.isEmpty()) {
-              break;
-          }
-          Cell colKv = results.get(LINK_TYPE_INDEX);
-          int colKeyLength = colKv.getRowLength();
-          PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength-offset);
-          int colKeyOffset = offset + colName.getBytes().length + 1;
-          PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength-colKeyOffset);
-          if (colName.getString().isEmpty() && famName != null) {
-              LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
-              if (linkType == LinkType.INDEX_TABLE) {
-                  addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes);
-              } else if (linkType == LinkType.PHYSICAL_TABLE) {
-                  physicalTables.add(famName);
-              } else if (linkType == LinkType.PARENT_TABLE) {
-                  parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
-                  parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
-              }
-          } else {
-              addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null);
-          }
+            results.clear();
+            scanner.next(results);
+            if (results.isEmpty()) {
+                break;
+            }
+            Cell colKv = results.get(LINK_TYPE_INDEX);
+            if (colKv != null) {
+                int colKeyLength = colKv.getRowLength();
+                PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength-offset);
+                int colKeyOffset = offset + colName.getBytes().length + 1;
+                PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength-colKeyOffset);
+                if (isQualifierCounterKV(colKv)) {
+                    Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
+                    cqCounter.setValue(famName.getString(), value);
+                } else {
+                    if (colName.getString().isEmpty() && famName != null) {
+                        LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
+                        if (linkType == LinkType.INDEX_TABLE) {
+                            addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes);
+                        } else if (linkType == LinkType.PHYSICAL_TABLE) {
+                            physicalTables.add(famName);
+                        } else if (linkType == LinkType.PARENT_TABLE) {
+                            parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
+                            parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
+                        }
+                    } else {
+                        addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null);
+                    }
+                } 
+            }
         }
         // Avoid querying the stats table because we're holding the rowLock here. Issuing an RPC to a remote
         // server while holding this lock is a bad idea and likely to cause contention.
@@ -964,9 +992,17 @@ 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);
+                indexDisableTimestamp, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, storageScheme, cqCounter);
     }
-
+    
+    private boolean isQualifierCounterKV(Cell kv) {
+        int cmp =
+                Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
+                    kv.getQualifierLength(), QUALIFIER_COUNTER_KV.getQualifierArray(),
+                    QUALIFIER_COUNTER_KV.getQualifierOffset(), QUALIFIER_COUNTER_KV.getQualifierLength());
+        return cmp == 0;
+    }
+    
     private PSchema getSchema(RegionScanner scanner, long clientTimeStamp) throws IOException, SQLException {
         List<Cell> results = Lists.newArrayList();
         scanner.next(results);
@@ -1486,46 +1522,46 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 Short indexId = null;
                 if (request.hasAllocateIndexId() && request.getAllocateIndexId()) {
                     String tenantIdStr = tenantIdBytes.length == 0 ? null : Bytes.toString(tenantIdBytes);
-                    try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)){
-                    PName physicalName = parentTable.getPhysicalName();
-                    int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
-                    SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
+                    try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) {
+                        PName physicalName = parentTable.getPhysicalName();
+                        int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
+                        SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
                             nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
                         // TODO Review Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
                         // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
                         // or not. 
-                    long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
-                    try {
-                        connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
+                        long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
+                        try {
+                            connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
                                 Short.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
-                    } catch (SequenceAlreadyExistsException e) {
-                    }
-                    long[] seqValues = new long[1];
-                    SQLException[] sqlExceptions = new SQLException[1];
-                    connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
+                        } catch (SequenceAlreadyExistsException e) {
+                        }
+                        long[] seqValues = new long[1];
+                        SQLException[] sqlExceptions = new SQLException[1];
+                        connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
                             HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
-                    if (sqlExceptions[0] != null) {
-                        throw sqlExceptions[0];
-                    }
-                    long seqValue = seqValues[0];
-                    if (seqValue > Short.MAX_VALUE) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        done.run(builder.build());
-                        return;
-                    }
-                    Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
-                    NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
-                    List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
-                    Cell cell = cells.get(0);
-                    PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
-                    Object val = dataType.toObject(seqValue, PLong.INSTANCE);
-                    byte[] bytes = new byte [dataType.getByteSize() + 1];
-                    dataType.toBytes(val, bytes, 0);
-                    Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
+                        if (sqlExceptions[0] != null) {
+                            throw sqlExceptions[0];
+                        }
+                        long seqValue = seqValues[0];
+                        if (seqValue > Short.MAX_VALUE) {
+                            builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
+                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                            done.run(builder.build());
+                            return;
+                        }
+                        Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
+                        NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
+                        List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
+                        Cell cell = cells.get(0);
+                        PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
+                        Object val = dataType.toObject(seqValue, PLong.INSTANCE);
+                        byte[] bytes = new byte [dataType.getByteSize() + 1];
+                        dataType.toBytes(val, bytes, 0);
+                        Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
                             cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
-                    cells.add(indexIdCell);
-                    indexId = (short) seqValue;
+                        cells.add(indexIdCell);
+                        indexId = (short) seqValue;
                     }
                 }
                 
@@ -1536,7 +1572,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // indexing on the system table. This is an issue because of the way we manage batch mutation
                 // in the Indexer.
                 region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
-
+                
                 // Invalidate the cache - the next getTable call will add it
                 // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
@@ -1988,7 +2024,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return result;
                 }
                 region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
-                // Invalidate from cache
+                // Invalidate from cache.
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
                 }
@@ -2161,6 +2197,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 byte[][] rkmd = new byte[5][];
                 int pkCount = getVarChars(m.getRow(), rkmd);
                 if (pkCount > COLUMN_NAME_INDEX
+                        && rkmd[COLUMN_NAME_INDEX] != null && rkmd[COLUMN_NAME_INDEX].length > 0
                         && Bytes.compareTo(schemaName, rkmd[SCHEMA_NAME_INDEX]) == 0
                         && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) == 0) {
                     columnPutsForBaseTable.add(new PutWithOrdinalPosition((Put)m, getInteger((Put)m, TABLE_FAMILY_BYTES, ORDINAL_POSITION_BYTES)));
@@ -2195,8 +2232,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
                 String columnFamily = rkmd[FAMILY_NAME_INDEX] == null ? null : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
                 try {
-                    existingViewColumn = columnFamily == null ? view.getColumn(columnName) : view.getColumnFamily(
-                            columnFamily).getColumn(columnName);
+                    existingViewColumn = columnFamily == null ? view.getPColumnForColumnName(columnName) : view.getColumnFamily(
+                            columnFamily).getPColumnForColumnName(columnName);
                 } catch (ColumnFamilyNotFoundException e) {
                     // ignore since it means that the column family is not present for the column to be added.
                 } catch (ColumnNotFoundException e) {
@@ -2323,26 +2360,26 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     columnsAddedToBaseTable++;
                 }
             }
-            /*
-             * Allow adding a pk columns to base table : 1. if all the view pk columns are exactly the same as the base
-             * table pk columns 2. if we are adding all the existing view pk columns to the base table
-             */ 
-            if (addingExistingPkCol && !viewPkCols.equals(basePhysicalTable.getPKColumns())) {
-                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-            }
-            addViewIndexesHeaderRowMutations(mutationsForAddingColumnsToViews, invalidateList, clientTimeStamp, view,
-                    deltaNumPkColsSoFar);
-            
-            /*
-             * Increment the sequence number by 1 if:
-             * 1) For a diverged view, there were columns (pk columns) added to the view.
-             * 2) For a non-diverged view if the base column count changed.
-             */
-            boolean changeSequenceNumber = (isDivergedView(view) && columnsAddedToView > 0)
-                    || (!isDivergedView(view) && columnsAddedToBaseTable > 0);
-            updateViewHeaderRow(basePhysicalTable, tableMetadata, mutationsForAddingColumnsToViews,
-                invalidateList, clientTimeStamp, columnsAddedToView, columnsAddedToBaseTable,
-                viewKey, view, ordinalPositionList, numCols, changeSequenceNumber);
+           /*
+            * Allow adding a pk columns to base table : 1. if all the view pk columns are exactly the same as the base
+            * table pk columns 2. if we are adding all the existing view pk columns to the base table
+            */ 
+           if (addingExistingPkCol && !viewPkCols.equals(basePhysicalTable.getPKColumns())) {
+               return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
+           }
+           addViewIndexesHeaderRowMutations(mutationsForAddingColumnsToViews, invalidateList, clientTimeStamp, view,
+                   deltaNumPkColsSoFar);
+           
+           /*
+            * Increment the sequence number by 1 if:
+            * 1) For a diverged view, there were columns (pk columns) added to the view.
+            * 2) For a non-diverged view if the base column count changed.
+            */
+           boolean changeSequenceNumber = (isDivergedView(view) && columnsAddedToView > 0)
+                   || (!isDivergedView(view) && columnsAddedToBaseTable > 0);
+           updateViewHeaderRow(basePhysicalTable, tableMetadata, mutationsForAddingColumnsToViews,
+               invalidateList, clientTimeStamp, columnsAddedToView, columnsAddedToBaseTable,
+               viewKey, view, ordinalPositionList, numCols, changeSequenceNumber);
         }
         return null;
     }
@@ -2500,8 +2537,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 byte[] columnKey = getColumnKey(viewKey, columnName, columnFamily);
                 try {
                     existingViewColumn =
-                            columnFamily == null ? view.getColumn(columnName) : view
-                                    .getColumnFamily(columnFamily).getColumn(columnName);
+                            columnFamily == null ? view.getPColumnForColumnName(columnName) : view
+                                    .getColumnFamily(columnFamily).getPColumnForColumnName(columnName);
                 } catch (ColumnFamilyNotFoundException e) {
                     // ignore since it means that the column family is not present for the column to
                     // be added.
@@ -2567,7 +2604,25 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     
     private MetaDataMutationResult validateColumnForAddToBaseTable(PColumn existingViewColumn, Put columnToBeAdded, PTable basePhysicalTable, boolean isColumnToBeAddPkCol, PTable view) {
         if (existingViewColumn != null) {
-            
+            if (EncodedColumnsUtil.usesEncodedColumnNames(basePhysicalTable) && !SchemaUtil.isPKColumn(existingViewColumn)) {
+                /*
+                 * If the column already exists in a view, then we cannot add the column to the base
+                 * table. The reason is subtle and is as follows: consider the case where a table
+                 * has two views where both the views have the same key value column KV. Now, we
+                 * dole out encoded column qualifiers for key value columns in views by using the
+                 * counters stored in the base physical table. So the KV column can have different
+                 * column qualifiers for the two views. For example, 11 for VIEW1 and 12 for VIEW2.
+                 * This naturally extends to rows being inserted using the two views having
+                 * different column qualifiers for the column named KV. Now, when an attempt is made
+                 * to add column KV to the base table, we cannot decide which column qualifier
+                 * should that column be assigned. It cannot be a number different than 11 or 12
+                 * since a query like SELECT KV FROM BASETABLE would return null for KV which is
+                 * incorrect since column KV is present in rows inserted from the two views. We
+                 * cannot use 11 or 12 either because we will then incorrectly return value of KV
+                 * column inserted using only one view.
+                 */
+                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
+            }
             // Validate data type is same
             int baseColumnDataType = getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES);
             if (baseColumnDataType != existingViewColumn.getDataType().getSqlType()) {
@@ -2797,6 +2852,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                 	return mutationResult;
                             } 
                         }
+                    } else if (type == PTableType.VIEW
+                            && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
+                        /*
+                         * When adding a column to a view that uses encoded column name scheme, we
+                         * need to modify the CQ counters stored in the view's physical table. So to
+                         * make sure clients get the latest PTable, we need to invalidate the cache
+                         * entry.
+                         */
+                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
+                                .getPhysicalTableRowForView(table)));
                     }
                     for (Mutation m : tableMetaData) {
                         byte[] key = m.getRow();
@@ -2810,7 +2875,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
                                     PColumnFamily family =
                                             table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
-                                    family.getColumn(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+                                    family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
                                 } else if (pkCount > COLUMN_NAME_INDEX
                                         && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
                                     addingPKColumn = true;
@@ -3063,7 +3128,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                         PColumnFamily family =
                                                 table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
                                         columnToDelete =
-                                                family.getColumn(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+                                                family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
                                     } else if (pkCount > COLUMN_NAME_INDEX
                                             && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
                                         deletePKColumn = true;
@@ -3152,10 +3217,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] indexKey =
                     SchemaUtil.getTableKey(tenantId, index.getSchemaName().getBytes(), index
                             .getTableName().getBytes());
+            Pair<String, String> columnToDeleteInfo = new Pair<>(columnToDelete.getFamilyName().getString(), columnToDelete.getName().getString());
+            boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(columnToDeleteInfo);
+            boolean isCoveredColumn = indexMaintainer.getCoveredColumnInfo().contains(columnToDeleteInfo);
             // If index requires this column for its pk, then drop it
-            if (indexMaintainer.getIndexedColumns().contains(
-                new ColumnReference(columnToDelete.getFamilyName().getBytes(), columnToDelete
-                        .getName().getBytes()))) {
+            if (isColumnIndexed) {
                 // Since we're dropping the index, lock it to ensure
                 // that a change in index state doesn't
                 // occur while we're dropping it.
@@ -3176,9 +3242,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 invalidateList.add(new ImmutableBytesPtr(indexKey));
             }
             // If the dropped column is a covered index column, invalidate the index
-            else if (indexMaintainer.getCoveredColumns().contains(
-                new ColumnReference(columnToDelete.getFamilyName().getBytes(), columnToDelete
-                        .getName().getBytes()))) {
+            else if (isCoveredColumn){
                 invalidateList.add(new ImmutableBytesPtr(indexKey));
             }
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
index 3cfe790..8a833ac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ScanRegionObserver.java
@@ -107,7 +107,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         }
     }
 
-    public static OrderedResultIterator deserializeFromScan(Scan scan, RegionScanner s) {
+    private static OrderedResultIterator deserializeFromScan(Scan scan, RegionScanner s) {
         byte[] topN = scan.getAttribute(BaseScannerRegionObserver.TOPN);
         if (topN == null) {
             return null;
@@ -125,7 +125,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                 orderByExpression.readFields(input);
                 orderByExpressions.add(orderByExpression);
             }
-            ResultIterator inner = new RegionScannerResultIterator(s);
+            ResultIterator inner = new RegionScannerResultIterator(s, ScanUtil.getMinMaxQualifiersFromScan(scan));
             return new OrderedResultIterator(inner, orderByExpressions, thresholdBytes, limit >= 0 ? limit : null, null,
                     estimatedRowSize);
         } catch (IOException e) {
@@ -218,21 +218,24 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        //TODO: samarth make this a client side check by looking at order by and group by expressions. Then use that to set min max qualifiers. We can then make useQualifierListAsIndex
+        // a member variable of BaseScannerRegionObserver.
+        boolean useQualifierAsIndex = ScanUtil.useQualifierAsIndex(ScanUtil.getMinMaxQualifiersFromScan(scan)) && scan.getAttribute(BaseScannerRegionObserver.TOPN) != null;
         innerScanner =
                 getWrappedScanner(c, innerScanner, arrayKVRefs, arrayFuncRefs, offset, scan,
                     dataColumns, tupleProjector, dataRegion, indexMaintainer, tx,
-                    viewConstants, kvSchema, kvSchemaBitSet, j == null ? p : null, ptr);
+                    viewConstants, kvSchema, kvSchemaBitSet, j == null ? p : null, ptr, useQualifierAsIndex);
 
         final ImmutableBytesPtr tenantId = ScanUtil.getTenantId(scan);
         if (j != null) {
-            innerScanner = new HashJoinRegionScanner(innerScanner, p, j, tenantId, c.getEnvironment());
+            innerScanner = new HashJoinRegionScanner(innerScanner, p, j, tenantId, c.getEnvironment(), useQualifierAsIndex);
         }
         if (scanOffset != null) {
             innerScanner = getOffsetScanner(c, innerScanner,
-                    new OffsetResultIterator(new RegionScannerResultIterator(innerScanner), scanOffset),
+                    new OffsetResultIterator(new RegionScannerResultIterator(innerScanner, ScanUtil.getMinMaxQualifiersFromScan(scan)), scanOffset),
                     scan.getAttribute(QueryConstants.LAST_SCAN) != null);
         }
-        final OrderedResultIterator iterator = deserializeFromScan(scan,innerScanner);
+        final OrderedResultIterator iterator = deserializeFromScan(scan, innerScanner);
         if (iterator == null) {
             return innerScanner;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
index 0d0f0c2..a313dd1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/UngroupedAggregateRegionObserver.java
@@ -24,6 +24,7 @@ import static org.apache.phoenix.query.QueryConstants.UNGROUPED_AGG_ROW_KEY;
 import static org.apache.phoenix.query.QueryServices.MUTATE_BATCH_SIZE_ATTRIB;
 import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.COMPACTION_UPDATE_STATS_ROW_COUNT;
 import static org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker.CONCURRENT_UPDATE_STATS_ROW_COUNT;
+import static org.apache.phoenix.util.ScanUtil.getMinMaxQualifiersFromScan;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -96,7 +97,10 @@ import org.apache.phoenix.schema.ValueSchema.Field;
 import org.apache.phoenix.schema.stats.StatisticsCollectionRunTracker;
 import org.apache.phoenix.schema.stats.StatisticsCollector;
 import org.apache.phoenix.schema.stats.StatisticsCollectorFactory;
+import org.apache.phoenix.schema.tuple.EncodedColumnQualiferCellsList;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.PositionBasedMultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PChar;
 import org.apache.phoenix.schema.types.PDataType;
@@ -305,6 +309,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         byte[] deleteCQ = null;
         byte[] deleteCF = null;
         byte[] emptyCF = null;
+        byte[] emptyKVQualifier = null;
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         if (upsertSelectTable != null) {
             isUpsert = true;
@@ -320,12 +325,14 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                 deleteCQ = scan.getAttribute(BaseScannerRegionObserver.DELETE_CQ);
             }
             emptyCF = scan.getAttribute(BaseScannerRegionObserver.EMPTY_CF);
+            emptyKVQualifier = scan.getAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER);
         }
         TupleProjector tupleProjector = null;
         byte[][] viewConstants = null;
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        boolean useQualifierAsIndex = ScanUtil.useQualifierAsIndex(ScanUtil.getMinMaxQualifiersFromScan(scan)) && scan.getAttribute(BaseScannerRegionObserver.TOPN) != null;
         if ((localIndexScan && !isDelete && !isDescRowKeyOrderUpgrade) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
@@ -334,11 +341,11 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
             theScanner =
                     getWrappedScanner(c, theScanner, offset, scan, dataColumns, tupleProjector, 
-                            c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
+                            c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr, useQualifierAsIndex);
         } 
         
         if (j != null)  {
-            theScanner = new HashJoinRegionScanner(theScanner, p, j, ScanUtil.getTenantId(scan), env);
+            theScanner = new HashJoinRegionScanner(theScanner, p, j, ScanUtil.getTenantId(scan), env, useQualifierAsIndex);
         }
         
         int batchSize = 0;
@@ -374,7 +381,8 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         Aggregator[] rowAggregators = aggregators.getAggregators();
         boolean hasMore;
         boolean hasAny = false;
-        MultiKeyValueTuple result = new MultiKeyValueTuple();
+        Pair<Integer, Integer> minMaxQualifiers = getMinMaxQualifiersFromScan(scan);
+        Tuple result = useQualifierAsIndex ? new PositionBasedMultiKeyValueTuple() : new MultiKeyValueTuple();
         if (logger.isDebugEnabled()) {
             logger.debug(LogUtil.addCustomAnnotations("Starting ungrouped coprocessor scan " + scan + " "+region.getRegionInfo(), ScanUtil.getCustomAnnotations(scan)));
         }
@@ -392,7 +400,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             acquiredLock = true;
             synchronized (innerScanner) {
                 do {
-                    List<Cell> results = new ArrayList<Cell>();
+                    List<Cell> results = useQualifierAsIndex ? new EncodedColumnQualiferCellsList(minMaxQualifiers.getFirst(), minMaxQualifiers.getSecond()) : new ArrayList<Cell>();
                     // Results are potentially returned even when the return value of s.next is false
                     // since this is an indication of whether or not there are more values after the
                     // ones returned


[03/13] phoenix git commit: PHOENIX-3461 Statistics collection broken if name space mapping enabled for SYSTEM tables

Posted by sa...@apache.org.
PHOENIX-3461 Statistics collection broken if name space mapping enabled for SYSTEM tables


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

Branch: refs/heads/encodecolumns2
Commit: d033abad232ea997f0890a63b8f9eaa3b1e7f5ce
Parents: b106347
Author: Samarth <sa...@salesforce.com>
Authored: Mon Nov 7 23:22:05 2016 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Nov 7 23:22:05 2016 -0800

----------------------------------------------------------------------
 .../apache/phoenix/schema/stats/DefaultStatisticsCollector.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d033abad/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
index c32efc8..3c375bb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/stats/DefaultStatisticsCollector.java
@@ -121,7 +121,8 @@ public class DefaultStatisticsCollector implements StatisticsCollector {
                     env.getRegion().getTableDesc());
         } else {
             // Next check for GUIDE_POST_WIDTH on table
-            HTableInterface htable = env.getTable(TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES));
+            HTableInterface htable = env.getTable(
+                    SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
             Get get = new Get(ptableKey);
             get.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH_BYTES);
             Result result = htable.get(get);


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

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
index 00ece40..15a9f74 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneExpressionVisitor.java
@@ -26,6 +26,7 @@ import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 import org.apache.phoenix.expression.DivideExpression;
 import org.apache.phoenix.expression.Expression;
@@ -80,6 +81,11 @@ public abstract class CloneExpressionVisitor extends TraverseAllExpressionVisito
     public Expression visit(KeyValueColumnExpression node) {
         return node;
     }
+    
+    @Override
+    public Expression visit(ArrayColumnExpression node) {
+        return node;
+    }
 
     @Override
     public Expression visit(ProjectedColumnExpression node) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
index 31f340d..100f099 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ExpressionVisitor.java
@@ -27,6 +27,7 @@ import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 import org.apache.phoenix.expression.DivideExpression;
 import org.apache.phoenix.expression.Expression;
@@ -113,6 +114,7 @@ public interface ExpressionVisitor<E> {
     public E visit(LiteralExpression node);
     public E visit(RowKeyColumnExpression node);
     public E visit(KeyValueColumnExpression node);
+    public E visit(ArrayColumnExpression node);
     public E visit(ProjectedColumnExpression node);
     public E visit(SequenceValueExpression node);
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseAllExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseAllExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseAllExpressionVisitor.java
index 3b7067a..9e50bc4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseAllExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseAllExpressionVisitor.java
@@ -26,9 +26,9 @@ import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 import org.apache.phoenix.expression.DivideExpression;
-import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.InListExpression;
 import org.apache.phoenix.expression.IsNullExpression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -121,6 +121,11 @@ public class StatelessTraverseAllExpressionVisitor<E> extends TraverseAllExpress
     }
     
     @Override
+    public E visit(ArrayColumnExpression node) {
+        return null;
+    }
+    
+    @Override
     public E visit(ProjectedColumnExpression node) {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseNoExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseNoExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseNoExpressionVisitor.java
index 83b28bd..1a2f2cc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseNoExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/StatelessTraverseNoExpressionVisitor.java
@@ -26,9 +26,9 @@ import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.CorrelateVariableFieldAccessExpression;
 import org.apache.phoenix.expression.DivideExpression;
-import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.InListExpression;
 import org.apache.phoenix.expression.IsNullExpression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -114,6 +114,11 @@ public class StatelessTraverseNoExpressionVisitor<E> extends TraverseNoExpressio
     public E visit(RowKeyColumnExpression node) {
         return null;
     }
+    
+    @Override
+    public E visit(ArrayColumnExpression node) {
+        return null;
+    }
 
     @Override
     public E visit(KeyValueColumnExpression node) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/filter/ColumnProjectionFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/ColumnProjectionFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/ColumnProjectionFilter.java
index 92e5c20..3d6843d 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/ColumnProjectionFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/ColumnProjectionFilter.java
@@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.util.Writables;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
-import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 
 /**
  * When selecting specific columns in a SELECT query, this filter passes only selected columns
@@ -54,6 +54,8 @@ public class ColumnProjectionFilter extends FilterBase implements Writable {
     private byte[] emptyCFName;
     private Map<ImmutableBytesPtr, NavigableSet<ImmutableBytesPtr>> columnsTracker;
     private Set<byte[]> conditionOnlyCfs;
+    private boolean usesEncodedColumnNames;
+    private byte[] emptyKVQualifier;
 
     public ColumnProjectionFilter() {
 
@@ -61,10 +63,12 @@ public class ColumnProjectionFilter extends FilterBase implements Writable {
 
     public ColumnProjectionFilter(byte[] emptyCFName,
             Map<ImmutableBytesPtr, NavigableSet<ImmutableBytesPtr>> columnsTracker,
-            Set<byte[]> conditionOnlyCfs) {
+            Set<byte[]> conditionOnlyCfs, boolean usesEncodedColumnNames) {
         this.emptyCFName = emptyCFName;
         this.columnsTracker = columnsTracker;
         this.conditionOnlyCfs = conditionOnlyCfs;
+        this.usesEncodedColumnNames = usesEncodedColumnNames;
+        this.emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(usesEncodedColumnNames).getFirst();
     }
 
     @Override
@@ -88,6 +92,9 @@ public class ColumnProjectionFilter extends FilterBase implements Writable {
             familyMapSize--;
         }
         int conditionOnlyCfsSize = WritableUtils.readVInt(input);
+        usesEncodedColumnNames = conditionOnlyCfsSize > 0;
+        emptyKVQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(usesEncodedColumnNames).getFirst();
+        conditionOnlyCfsSize = Math.abs(conditionOnlyCfsSize) - 1; // restore to the actual value.
         this.conditionOnlyCfs = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
         while (conditionOnlyCfsSize > 0) {
             this.conditionOnlyCfs.add(WritableUtils.readCompressedByteArray(input));
@@ -111,12 +118,13 @@ public class ColumnProjectionFilter extends FilterBase implements Writable {
                 }
             }
         }
-        // Write conditionOnlyCfs
-        WritableUtils.writeVInt(output, this.conditionOnlyCfs.size());
+        // Encode usesEncodedColumnNames in conditionOnlyCfs size.
+        WritableUtils.writeVInt(output, (this.conditionOnlyCfs.size() + 1) * (usesEncodedColumnNames ? 1 : -1));
         for (byte[] f : this.conditionOnlyCfs) {
             WritableUtils.writeCompressedByteArray(output, f);
         }
-    }
+    
+}
 
     @Override
     public byte[] toByteArray() throws IOException {
@@ -156,9 +164,9 @@ public class ColumnProjectionFilter extends FilterBase implements Writable {
         // make sure we're not holding to any of the byte[]'s
         ptr.set(HConstants.EMPTY_BYTE_ARRAY);
         if (kvs.isEmpty()) {
-            kvs.add(new KeyValue(firstKV.getRowArray(), firstKV.getRowOffset(),firstKV.getRowLength(), this.emptyCFName,
-                    0, this.emptyCFName.length, QueryConstants.EMPTY_COLUMN_BYTES, 0,
-                    QueryConstants.EMPTY_COLUMN_BYTES.length, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0));
+            kvs.add(new KeyValue(firstKV.getRowArray(), firstKV.getRowOffset(), firstKV.getRowLength(),
+                    this.emptyCFName, 0, this.emptyCFName.length, emptyKVQualifier, 0,
+                    emptyKVQualifier.length, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiKeyValueComparisonFilter.java
index dba700b..a7146fc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiKeyValueComparisonFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiKeyValueComparisonFilter.java
@@ -26,6 +26,7 @@ import java.util.TreeSet;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
@@ -94,7 +95,7 @@ public abstract class MultiKeyValueComparisonFilter extends BooleanExpressionFil
             refCount = foundColumns.size();
         }
         
-        public ReturnCode resolveColumn(Cell value) {
+        private ReturnCode resolveColumn(Cell value) {
             // Always set key, in case we never find a key value column of interest,
             // and our expression uses row key columns.
             setKey(value);
@@ -184,7 +185,7 @@ public abstract class MultiKeyValueComparisonFilter extends BooleanExpressionFil
         ExpressionVisitor<Void> visitor = new StatelessTraverseAllExpressionVisitor<Void>() {
             @Override
             public Void visit(KeyValueColumnExpression expression) {
-                inputTuple.addColumn(expression.getColumnFamily(), expression.getColumnName());
+                inputTuple.addColumn(expression.getColumnFamily(), expression.getColumnQualifier());
                 return null;
             }
         };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCQKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCQKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCQKeyValueComparisonFilter.java
index 0d904bc..195c89c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCQKeyValueComparisonFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleCQKeyValueComparisonFilter.java
@@ -47,7 +47,8 @@ public class SingleCQKeyValueComparisonFilter extends SingleKeyValueComparisonFi
 
     public static SingleCQKeyValueComparisonFilter parseFrom(final byte [] pbBytes) throws DeserializationException {
         try {
-            return (SingleCQKeyValueComparisonFilter)Writables.getWritable(pbBytes, new SingleCQKeyValueComparisonFilter());
+            SingleCQKeyValueComparisonFilter writable = (SingleCQKeyValueComparisonFilter)Writables.getWritable(pbBytes, new SingleCQKeyValueComparisonFilter());
+            return writable;
         } catch (IOException e) {
             throw new DeserializationException(e);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleKeyValueComparisonFilter.java
index eaf8d35..b97c4e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleKeyValueComparisonFilter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/SingleKeyValueComparisonFilter.java
@@ -22,11 +22,13 @@ import java.io.IOException;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisitor;
 import org.apache.phoenix.expression.visitor.TraverseAllExpressionVisitor;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
 
 
 
@@ -58,7 +60,7 @@ public abstract class SingleKeyValueComparisonFilter extends BooleanExpressionFi
             @Override
             public Void visit(KeyValueColumnExpression expression) {
                 cf = expression.getColumnFamily();
-                cq = expression.getColumnName();
+                cq = expression.getColumnQualifier();
                 return null;
             }
         };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
index bcadc2b..19797cf 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/ValueGetter.java
@@ -35,4 +35,5 @@ public interface ValueGetter {
   public ImmutableBytesWritable getLatestValue(ColumnReference ref) throws IOException;
   
   public byte[] getRowKey();
+  
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
index 6f9caa6..0f960e4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/covered/example/CoveredColumnIndexCodec.java
@@ -32,7 +32,6 @@ import org.apache.phoenix.hbase.index.covered.TableState;
 import org.apache.phoenix.hbase.index.scanner.Scanner;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Lists;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/KeyValueBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/KeyValueBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/KeyValueBuilder.java
index 741bf87..56b60e9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/KeyValueBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/hbase/index/util/KeyValueBuilder.java
@@ -125,4 +125,5 @@ public abstract class KeyValueBuilder {
   public abstract KVComparator getKeyValueComparator();
   
   public abstract List<Mutation> cloneIfNecessary(List<Mutation> mutations);
+  
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 237ed75..7c88a25 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
@@ -17,12 +17,15 @@
  */
 package org.apache.phoenix.index;
 
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
+
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -32,6 +35,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
@@ -44,16 +48,20 @@ import org.apache.hadoop.hbase.client.Durability;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.IndexExpressionCompiler;
 import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.visitor.KeyValueExpressionVisitor;
 import org.apache.phoenix.hbase.index.ValueGetter;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
@@ -67,14 +75,16 @@ import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
 import org.apache.phoenix.parse.UDFParseNode;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.AmbiguousColumnException;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PIndexState;
-import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
@@ -82,10 +92,13 @@ import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.ValueSchema;
 import org.apache.phoenix.schema.ValueSchema.Field;
+import org.apache.phoenix.schema.tuple.BaseTuple;
 import org.apache.phoenix.schema.tuple.ValueGetterTuple;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.BitSet;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
@@ -93,6 +106,7 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 import org.apache.tephra.TxConstants;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
@@ -276,8 +290,14 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     // columns required to evaluate all expressions in indexedExpressions (this does not include columns in the data row key)
     private Set<ColumnReference> indexedColumns;
     private Set<ColumnReference> coveredColumns;
-    // Map used to cache column family of data table and the corresponding column family for the local index
-    private Map<ImmutableBytesPtr, ImmutableBytesWritable> dataTableLocalIndexFamilyMap;
+    // Information for columns of data tables that are being indexed. The first part of the pair is column family and second part is the column name. 
+    private Set<Pair<String, String>> indexedColumnsInfo;
+    // Information for columns of data tables that are being covered by the index. The first part of the pair is column family and second part is the column name.
+    private Set<Pair<String, String>> coveredColumnsInfo;
+    // Map of covered columns where a key is column reference for a column in the data table
+    // and value is column reference for corresponding column in the index table.
+    // TODO: samarth confirm that we don't need a separate map for tracking column families of local indexes.
+    private Map<ColumnReference, ColumnReference> coveredColumnsMap;
     // columns required to create index row i.e. indexedColumns + coveredColumns  (this does not include columns in the data row key)
     private Set<ColumnReference> allColumns;
     // TODO remove this in the next major release
@@ -291,39 +311,40 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     private boolean indexWALDisabled;
     private boolean isLocalIndex;
     private boolean immutableRows;
+    private boolean storeColsInSingleCell;
 
     // Transient state
     private final boolean isDataTableSalted;
     private final RowKeySchema dataRowKeySchema;
     
-    private List<ImmutableBytesPtr> indexQualifiers;
     private int estimatedIndexRowKeyBytes;
     private int estimatedExpressionSize;
     private int[] dataPkPosition;
     private int maxTrailingNulls;
     private ColumnReference dataEmptyKeyValueRef;
     private boolean rowKeyOrderOptimizable;
+    private boolean usesEncodedColumnNames;
+    private ImmutableBytesPtr emptyKeyValueQualifierPtr;
     
     private IndexMaintainer(RowKeySchema dataRowKeySchema, boolean isDataTableSalted) {
         this.dataRowKeySchema = dataRowKeySchema;
         this.isDataTableSalted = isDataTableSalted;
     }
 
-    private IndexMaintainer(PTable dataTable, PTable index, PhoenixConnection connection) {
+    private IndexMaintainer(final PTable dataTable, final PTable index, PhoenixConnection connection) {
         this(dataTable.getRowKeySchema(), dataTable.getBucketNum() != null);
         assert(dataTable.getType() == PTableType.SYSTEM || dataTable.getType() == PTableType.TABLE || dataTable.getType() == PTableType.VIEW);
         this.rowKeyOrderOptimizable = index.rowKeyOrderOptimizable();
         this.isMultiTenant = dataTable.isMultiTenant();
         this.viewIndexId = index.getViewIndexId() == null ? null : MetaDataUtil.getViewIndexIdDataType().toBytes(index.getViewIndexId());
         this.isLocalIndex = index.getIndexType() == IndexType.LOCAL;
-
+        this.usesEncodedColumnNames = EncodedColumnsUtil.usesEncodedColumnNames(index);
         byte[] indexTableName = index.getPhysicalName().getBytes();
         // Use this for the nDataSaltBuckets as we need this for local indexes
         // TODO: persist nDataSaltBuckets separately, but maintain b/w compat.
         Integer nIndexSaltBuckets = isLocalIndex ? dataTable.getBucketNum() : index.getBucketNum();
         boolean indexWALDisabled = index.isWALDisabled();
         int indexPosOffset = (index.getBucketNum() == null ? 0 : 1) + (this.isMultiTenant ? 1 : 0) + (this.viewIndexId == null ? 0 : 1);
-//        int indexPosOffset = !isLocalIndex && nIndexSaltBuckets > 0 ? 1 : 0;
         int nIndexColumns = index.getColumns().size() - indexPosOffset;
         int nIndexPKColumns = index.getPKColumns().size() - indexPosOffset;
         // number of expressions that are indexed that are not present in the row key of the data table
@@ -334,7 +355,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             String dataFamilyName = IndexUtil.getDataColumnFamilyName(indexColumnName);
             String dataColumnName = IndexUtil.getDataColumnName(indexColumnName);
             try {
-                PColumn dataColumn = dataFamilyName.equals("") ? dataTable.getColumn(dataColumnName) : dataTable.getColumnFamily(dataFamilyName).getColumn(dataColumnName);
+                PColumn dataColumn = dataFamilyName.equals("") ? dataTable.getPColumnForColumnName(dataColumnName) : dataTable.getColumnFamily(dataFamilyName).getPColumnForColumnName(dataColumnName);
                 if (SchemaUtil.isPKColumn(dataColumn)) 
                     continue;
             } catch (ColumnNotFoundException e) {
@@ -367,7 +388,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         this.indexedColumnTypes = Lists.<PDataType>newArrayListWithExpectedSize(nIndexPKColumns-nDataPKColumns);
         this.indexedExpressions = Lists.newArrayListWithExpectedSize(nIndexPKColumns-nDataPKColumns);
         this.coveredColumns = Sets.newLinkedHashSetWithExpectedSize(nIndexColumns-nIndexPKColumns);
-        this.dataTableLocalIndexFamilyMap = Maps.newHashMapWithExpectedSize(nIndexColumns-nIndexPKColumns);
+        this.coveredColumnsMap = Maps.newHashMapWithExpectedSize(nIndexColumns - nIndexPKColumns);
         this.nIndexSaltBuckets  = nIndexSaltBuckets == null ? 0 : nIndexSaltBuckets;
         this.dataEmptyKeyValueCF = SchemaUtil.getEmptyColumnFamily(dataTable);
         this.emptyKeyValueCFPtr = SchemaUtil.getEmptyColumnFamilyPtr(index);
@@ -376,6 +397,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         // TODO: check whether index is immutable or not. Currently it's always false so checking
         // data table is with immutable rows or not.
         this.immutableRows = dataTable.isImmutableRows();
+        this.storeColsInSingleCell = index.getStorageScheme() == StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL;
         int indexColByteSize = 0;
         ColumnResolver resolver = null;
         List<ParseNode> parseNodes = new ArrayList<ParseNode>(1);
@@ -397,6 +419,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             throw new RuntimeException(e); // Impossible
         }
         StatementContext context = new StatementContext(new PhoenixStatement(connection), resolver);
+        this.indexedColumnsInfo = Sets.newHashSetWithExpectedSize(nIndexColumns - nIndexPKColumns);
+        this.coveredColumnsInfo = Sets.newHashSetWithExpectedSize(nIndexColumns - nIndexPKColumns);
+        
         IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
         for (int i = indexPosOffset; i < index.getPKColumns().size(); i++) {
             PColumn indexColumn = index.getPKColumns().get(i);
@@ -409,12 +434,13 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                 throw new RuntimeException(e); // Impossible
             }
             if ( expressionIndexCompiler.getColumnRef()!=null ) {
-            	// get the column of the data table that corresponds to this index column
+            	// get the column of the data column that corresponds to this index column
 	            PColumn column = IndexUtil.getDataColumn(dataTable, indexColumn.getName().getString());
 	            boolean isPKColumn = SchemaUtil.isPKColumn(column);
 	            if (isPKColumn) {
 	                int dataPkPos = dataTable.getPKColumns().indexOf(column) - (dataTable.getBucketNum() == null ? 0 : 1) - (this.isMultiTenant ? 1 : 0);
 	                this.rowKeyMetaData.setIndexPkPosition(dataPkPos, indexPos);
+	                indexedColumnsInfo.add(new Pair<>((String)null, column.getName().getString()));
 	            } else {
 	                indexColByteSize += column.getDataType().isFixedWidth() ? SchemaUtil.getFixedByteSize(column) : ValueSchema.ESTIMATED_VARIABLE_LENGTH_SIZE;
 	                try {
@@ -424,6 +450,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
 	                        expression = CoerceExpression.create(expression, indexColumn.getDataType());
 	                    }
                         this.indexedExpressions.add(expression);
+                        indexedColumnsInfo.add(new Pair<>(column.getFamilyName().getString(), column.getName().getString()));
                     } catch (SQLException e) {
                         throw new RuntimeException(e); // Impossible
                     }
@@ -432,6 +459,45 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             else {
             	indexColByteSize += expression.getDataType().isFixedWidth() ? SchemaUtil.getFixedByteSize(expression) : ValueSchema.ESTIMATED_VARIABLE_LENGTH_SIZE;
                 this.indexedExpressions.add(expression);
+                KeyValueExpressionVisitor kvVisitor = new KeyValueExpressionVisitor() {
+                    @Override
+                    public Void visit(KeyValueColumnExpression colExpression) {
+                        return addDataColInfo(dataTable, colExpression);
+                    }
+
+                    @Override
+                    public Void visit(ArrayColumnExpression expression) {
+                        return addDataColInfo(dataTable, expression);
+                    }
+
+                    private Void addDataColInfo(final PTable dataTable, Expression expression) {
+                        Preconditions.checkArgument(expression instanceof ArrayColumnExpression
+                                || expression instanceof KeyValueColumnExpression);
+
+                        KeyValueColumnExpression colExpression = null;
+                        if (expression instanceof ArrayColumnExpression) {
+                            colExpression =
+                                    ((ArrayColumnExpression) expression).getKeyValueExpression();
+                        } else {
+                            colExpression = ((KeyValueColumnExpression) expression);
+                        }
+                        byte[] cf = colExpression.getColumnFamily();
+                        byte[] cq = colExpression.getColumnQualifier();
+                        try {
+                            PColumn dataColumn =
+                                    cf == null ? dataTable.getPColumnForColumnQualifier(null, cq)
+                                            : dataTable.getColumnFamily(cf)
+                                                    .getPColumnForColumnQualifier(cq);
+                            indexedColumnsInfo.add(new Pair<>(dataColumn.getFamilyName()
+                                    .getString(), dataColumn.getName().getString()));
+                        } catch (ColumnNotFoundException | ColumnFamilyNotFoundException
+                                | AmbiguousColumnException e) {
+                            throw new RuntimeException(e);
+                        }
+                        return null;
+                    }
+                };
+                expression.accept(kvVisitor);
             }
             // set the sort order of the expression correctly
             if (indexColumn.getSortOrder() == SortOrder.DESC) {
@@ -442,18 +508,19 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         for (int i = 0; i < index.getColumnFamilies().size(); i++) {
             PColumnFamily family = index.getColumnFamilies().get(i);
             for (PColumn indexColumn : family.getColumns()) {
-                PColumn column = IndexUtil.getDataColumn(dataTable, indexColumn.getName().getString());
-                PName dataTableFamily = column.getFamilyName();
-                this.coveredColumns.add(new ColumnReference(dataTableFamily.getBytes(), column.getName().getBytes()));
-                if(isLocalIndex) {
-                    this.dataTableLocalIndexFamilyMap.put(new ImmutableBytesPtr(dataTableFamily.getBytes()), new ImmutableBytesWritable(Bytes.toBytes(IndexUtil.getLocalIndexColumnFamily(dataTableFamily.getString()))));
-                }
+                PColumn dataColumn = IndexUtil.getDataColumn(dataTable, indexColumn.getName().getString());
+                byte[] dataColumnCq = EncodedColumnsUtil.getColumnQualifier(dataColumn, dataTable);
+                byte[] indexColumnCq = EncodedColumnsUtil.getColumnQualifier(indexColumn, index);
+                this.coveredColumns.add(new ColumnReference(dataColumn.getFamilyName().getBytes(), dataColumnCq));
+                this.coveredColumnsMap.put(new ColumnReference(dataColumn.getFamilyName().getBytes(), dataColumnCq), 
+                        new ColumnReference(indexColumn.getFamilyName().getBytes(), indexColumnCq));
+                this.coveredColumnsInfo.add(new Pair<>(dataColumn.getFamilyName().getString(), dataColumn.getName().getString()));
             }
         }
         this.estimatedIndexRowKeyBytes = estimateIndexRowKeyByteSize(indexColByteSize);
         initCachedState();
     }
-
+    
     public byte[] buildRowKey(ValueGetter valueGetter, ImmutableBytesWritable rowKeyPtr, byte[] regionStartKey, byte[] regionEndKey)  {
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         boolean prependRegionStartKey = isLocalIndex && regionStartKey != null;
@@ -856,35 +923,113 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     }
 
     public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] regionStartKey, byte[] regionEndKey) throws IOException {
-        Put put = null;
+    	byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey);
+    	Put put = null;
         // New row being inserted: add the empty key value
         if (valueGetter.getLatestValue(dataEmptyKeyValueRef) == null) {
-            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey);
             put = new Put(indexRowKey);
             // add the keyvalue for the empty row
             put.add(kvBuilder.buildPut(new ImmutableBytesPtr(indexRowKey),
-                this.getEmptyKeyValueFamily(), QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts,
+                this.getEmptyKeyValueFamily(), emptyKeyValueQualifierPtr, ts,
                 // set the value to the empty column name
-                QueryConstants.EMPTY_COLUMN_BYTES_PTR));
+                emptyKeyValueQualifierPtr));
             put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
         }
-        int i = 0;
-        for (ColumnReference ref : this.getCoveredColumns()) {
-            ImmutableBytesPtr cq = this.indexQualifiers.get(i++);
-            ImmutableBytesWritable value = valueGetter.getLatestValue(ref);
-            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey);
-            ImmutableBytesPtr rowKey = new ImmutableBytesPtr(indexRowKey);
-            if (value != null) {
+        ImmutableBytesPtr rowKey = new ImmutableBytesPtr(indexRowKey);
+        if (storeColsInSingleCell) {
+            // map from index column family to list of pair of index column and data column (for covered columns)
+            Map<ByteBuffer, List<Pair<ColumnReference, ColumnReference>>> familyToColListMap = Maps.newHashMap();
+            for (ColumnReference ref : this.getCoveredColumns()) {
+            	ColumnReference indexColRef = this.coveredColumnsMap.get(ref);
+                ByteBuffer cf = ByteBuffer.wrap(indexColRef.getFamily());
+                if (!familyToColListMap.containsKey(cf)) {
+                    familyToColListMap.put(cf, Lists.<Pair<ColumnReference, ColumnReference>>newArrayList());
+                }
+                familyToColListMap.get(cf).add(Pair.newPair(indexColRef, ref));
+            }
+            // iterate over each column family and create a byte[] containing all the columns 
+            for (Entry<ByteBuffer, List<Pair<ColumnReference, ColumnReference>>> entry : familyToColListMap.entrySet()) {
+                byte[] columnFamily = entry.getKey().array();
+                List<Pair<ColumnReference, ColumnReference>> colRefPairs = entry.getValue();
+                int maxIndex = Integer.MIN_VALUE;
+                // find the max col qualifier
+                for (Pair<ColumnReference, ColumnReference> colRefPair : colRefPairs) {
+                    int qualifier = getEncodedColumnQualifier(colRefPair.getFirst().getQualifier());
+                    maxIndex = Math.max(maxIndex, qualifier);
+                }
+                byte[][] colValues = new byte[maxIndex+1][];
+                // set the values of the columns
+                for (Pair<ColumnReference, ColumnReference> colRefPair : colRefPairs) {
+                	ColumnReference indexColRef = colRefPair.getFirst();
+                	ColumnReference dataColRef = colRefPair.getSecond();
+                	int dataArrayPos = getEncodedColumnQualifier(dataColRef.getQualifier());
+                	Expression expression = new ArrayColumnExpression(new PDatum() {
+						@Override
+						public boolean isNullable() {
+							return false;
+						}
+						
+						@Override
+						public SortOrder getSortOrder() {
+							return null;
+						}
+						
+						@Override
+						public Integer getScale() {
+							return null;
+						}
+						
+						@Override
+						public Integer getMaxLength() {
+							return null;
+						}
+						
+						@Override
+						public PDataType getDataType() {
+							return null;
+						}
+					}, dataColRef.getFamily(), dataArrayPos);
+                	ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+                    expression.evaluate(new ValueGetterTuple(valueGetter), ptr);
+                    byte[] value = ptr.copyBytesIfNecessary();
+					if (value != null) {
+						int indexArrayPos = getEncodedColumnQualifier(indexColRef.getQualifier());
+                        colValues[indexArrayPos] = value;
+                    }
+                }
+                
+                List<Expression> children = Lists.newArrayListWithExpectedSize(colRefPairs.size());
+                // create an expression list with all the columns
+                for (int j=0; j<colValues.length; ++j) {
+                    children.add(new LiteralExpression(colValues[j]==null ? ByteUtil.EMPTY_BYTE_ARRAY : colValues[j] ));
+                }
+                // we use ArrayConstructorExpression to serialize multiple columns into a single byte[]
+                // construct the ArrayConstructorExpression with a variable length data type (PVarchar) since columns can be of fixed or variable length 
+                ArrayConstructorExpression arrayExpression = new ArrayConstructorExpression(children, PVarchar.INSTANCE, rowKeyOrderOptimizable);
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                arrayExpression.evaluate(new BaseTuple() {}, ptr);
                 if (put == null) {
                     put = new Put(indexRowKey);
                     put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
                 }
+                ImmutableBytesPtr colFamilyPtr = new ImmutableBytesPtr(columnFamily);
                 //this is a little bit of extra work for installations that are running <0.94.14, but that should be rare and is a short-term set of wrappers - it shouldn't kill GC
-                if(this.isLocalIndex) {
-                    ImmutableBytesWritable localIndexColFamily = this.dataTableLocalIndexFamilyMap.get(ref.getFamilyWritable());
-                    put.add(kvBuilder.buildPut(rowKey, localIndexColFamily, cq, ts, value));
-                } else {
-                    put.add(kvBuilder.buildPut(rowKey, ref.getFamilyWritable(), cq, ts, value));
+                put.add(kvBuilder.buildPut(rowKey, colFamilyPtr, colFamilyPtr, ts, ptr));
+            }
+        }
+        else {
+        	for (ColumnReference ref : this.getCoveredColumns()) {
+                //FIXME: samarth figure out a backward compatible way to handle this as coveredcolumnsmap won't be availble for older phoenix clients.
+                ColumnReference indexColRef = this.coveredColumnsMap.get(ref);
+                ImmutableBytesPtr cq = indexColRef.getQualifierWritable();
+                ImmutableBytesPtr cf = indexColRef.getFamilyWritable();
+                ImmutableBytesWritable value = valueGetter.getLatestValue(ref);
+                if (value != null) {
+                	if (put == null) {
+                        put = new Put(indexRowKey);
+                        put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
+                    }
+                	put.add(kvBuilder.buildPut(rowKey, cf, cq, ts, value));
                 }
             }
         }
@@ -973,14 +1118,12 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             
             for (ColumnReference ref : getCoveredColumns()) {
                 byte[] family = ref.getFamily();
-                if (this.isLocalIndex) {
-                    family = this.dataTableLocalIndexFamilyMap.get(ref.getFamilyWritable()).get();
-                }
+                ColumnReference indexColumn = coveredColumnsMap.get(ref);
                 // If table delete was single version, then index delete should be as well
                 if (deleteType == DeleteType.SINGLE_VERSION) {
-                    delete.deleteFamilyVersion(family, ts);
+                    delete.deleteFamilyVersion(indexColumn.getFamily(), ts);
                 } else {
-                    delete.deleteFamily(family, ts);
+                    delete.deleteFamily(indexColumn.getFamily(), ts);
                 }
             }
             if (deleteType == DeleteType.SINGLE_VERSION) {
@@ -1001,12 +1144,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                         delete = new Delete(indexRowKey);                    
                         delete.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
                     }
-                    byte[] family = this.isLocalIndex ? this.dataTableLocalIndexFamilyMap.get(ref.getFamilyWritable()).get() : ref.getFamily();
+                    ColumnReference indexColumn = coveredColumnsMap.get(ref);
                     // If point delete for data table, then use point delete for index as well
-                    if (kv.getTypeByte() == KeyValue.Type.Delete.getCode()) {
-                        delete.deleteColumn(family, IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
+                    if (kv.getTypeByte() == KeyValue.Type.Delete.getCode()) { 
+                        //FIXME: samarth change this. Index column qualifiers are not derivable from data table cqs.
+                        // Figure out a backward compatible way of going this since coveredColumnsMap won't be available
+                        // for older clients.
+                        delete.deleteColumn(indexColumn.getFamily(), indexColumn.getQualifier(), ts);
                     } else {
-                        delete.deleteColumns(family, IndexUtil.getIndexColumnName(ref.getFamily(), ref.getQualifier()), ts);
+                        delete.deleteColumns(indexColumn.getFamily(), indexColumn.getQualifier(), ts);
                     }
                 }
             }
@@ -1061,15 +1207,16 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         isLocalIndex = encodedCoveredolumnsAndLocalIndex < 0;
         int nCoveredColumns = Math.abs(encodedCoveredolumnsAndLocalIndex) - 1;
         coveredColumns = Sets.newLinkedHashSetWithExpectedSize(nCoveredColumns);
-        dataTableLocalIndexFamilyMap = Maps.newHashMapWithExpectedSize(nCoveredColumns);
+        coveredColumnsMap = Maps.newHashMapWithExpectedSize(nCoveredColumns);
         for (int i = 0; i < nCoveredColumns; i++) {
-            byte[] cf = Bytes.readByteArray(input);
-            byte[] cq = Bytes.readByteArray(input);
-            ColumnReference ref = new ColumnReference(cf,cq);
-            coveredColumns.add(ref);
-            if(isLocalIndex) {
-                dataTableLocalIndexFamilyMap.put(ref.getFamilyWritable(), new ImmutableBytesWritable(Bytes.toBytes(IndexUtil.getLocalIndexColumnFamily(Bytes.toString(cf)))));
-            }
+            byte[] dataTableCf = Bytes.readByteArray(input);
+            byte[] dataTableCq = Bytes.readByteArray(input);
+            byte[] indexTableCf = Bytes.readByteArray(input);
+            byte[] indexTableCq = Bytes.readByteArray(input);
+            ColumnReference dataColumn = new ColumnReference(dataTableCf, dataTableCq); 
+            coveredColumns.add(dataColumn);
+            ColumnReference indexColumn = new ColumnReference(indexTableCf, indexTableCq);
+            coveredColumnsMap.put(dataColumn, indexColumn);
         }
         // Hack to serialize whether the index row key is optimizable
         int len = WritableUtils.readVInt(input);
@@ -1095,6 +1242,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         
         if (isNewClient) {
             int numIndexedExpressions = WritableUtils.readVInt(input);
+            usesEncodedColumnNames = numIndexedExpressions > 0;
+            numIndexedExpressions = Math.abs(numIndexedExpressions) - 1;
             indexedExpressions = Lists.newArrayListWithExpectedSize(numIndexedExpressions);        
             for (int i = 0; i < numIndexedExpressions; i++) {
             	Expression expression = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
@@ -1148,6 +1297,22 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         int encodedEstimatedIndexRowKeyBytesAndImmutableRows = WritableUtils.readVInt(input);
         this.immutableRows = encodedEstimatedIndexRowKeyBytesAndImmutableRows < 0;
         this.estimatedIndexRowKeyBytes = Math.abs(encodedEstimatedIndexRowKeyBytesAndImmutableRows);
+        int numCols = WritableUtils.readVInt(input);
+        //TODO: samarth figure out a backward compatible way of reading/writing indexedColumnsInfo
+        indexedColumnsInfo = Sets.newHashSetWithExpectedSize(numCols);
+        for (int i = 1; i <= numCols; i++) {
+            byte[] dataTableCf = Bytes.readByteArray(input);
+            byte[] dataTableCq = Bytes.readByteArray(input);
+            indexedColumnsInfo.add(new Pair<>(Bytes.toString(dataTableCf), Bytes.toString(dataTableCq)));
+        }
+        coveredColumnsInfo = Sets.newHashSetWithExpectedSize(numCols);
+        int numCoveredCols = WritableUtils.readVInt(input);
+        for (int i = 1; i <= numCoveredCols; i++) {
+            byte[] dataTableCf = Bytes.readByteArray(input);
+            byte[] dataTableCq = Bytes.readByteArray(input);
+            coveredColumnsInfo.add(new Pair<>(Bytes.toString(dataTableCf), Bytes.toString(dataTableCq)));
+        }
+        storeColsInSingleCell = WritableUtils.readVInt(input) > 0;
         initCachedState();
     }
     
@@ -1171,9 +1336,13 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         }
         // Encode coveredColumns.size() and whether or not this is a local index
         WritableUtils.writeVInt(output, (coveredColumns.size() + 1) * (isLocalIndex ? -1 : 1));
-        for (ColumnReference ref : coveredColumns) {
-            Bytes.writeByteArray(output, ref.getFamily());
-            Bytes.writeByteArray(output, ref.getQualifier());
+        for (Entry<ColumnReference, ColumnReference> ref : coveredColumnsMap.entrySet()) {
+            ColumnReference dataColumn = ref.getKey();
+            ColumnReference indexColumn = ref.getValue();
+            Bytes.writeByteArray(output, dataColumn.getFamily());
+            Bytes.writeByteArray(output, dataColumn.getQualifier());
+            Bytes.writeByteArray(output, indexColumn.getFamily());
+            Bytes.writeByteArray(output, indexColumn.getQualifier());
         }
         // TODO: remove when rowKeyOrderOptimizable hack no longer needed
         WritableUtils.writeVInt(output,indexTableName.length * (rowKeyOrderOptimizable ? 1 : -1));
@@ -1184,7 +1353,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         WritableUtils.writeVInt(output,-emptyKeyValueCFPtr.getLength());
         output.write(emptyKeyValueCFPtr.get(),emptyKeyValueCFPtr.getOffset(), emptyKeyValueCFPtr.getLength());
         
-        WritableUtils.writeVInt(output, indexedExpressions.size());
+        // Hack to encode usesEncodedColumnNames in indexedExpressions size.
+        int indexedExpressionsSize = (indexedExpressions.size() + 1) * (usesEncodedColumnNames ? 1 : -1);
+        WritableUtils.writeVInt(output, indexedExpressionsSize);
         for (Expression expression : indexedExpressions) {
         	WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
         	expression.write(output);
@@ -1195,6 +1366,17 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         WritableUtils.writeVInt(output, (nDataCFs + 1) * (indexWALDisabled ? -1 : 1));
         // Encode estimatedIndexRowKeyBytes and immutableRows together.
         WritableUtils.writeVInt(output, estimatedIndexRowKeyBytes * (immutableRows ? -1 : 1));
+        WritableUtils.writeVInt(output, indexedColumnsInfo.size());
+        for (Pair<String, String> colInfo : indexedColumnsInfo) {
+            Bytes.writeByteArray(output, colInfo.getFirst() == null ? null : colInfo.getFirst().getBytes());
+            Bytes.writeByteArray(output, colInfo.getSecond().getBytes());
+        }
+        WritableUtils.writeVInt(output, coveredColumnsInfo.size());
+        for (Pair<String, String> colInfo : coveredColumnsInfo) {
+            Bytes.writeByteArray(output, colInfo.getFirst() == null ? null : colInfo.getFirst().getBytes());
+            Bytes.writeByteArray(output, colInfo.getSecond().getBytes());
+        }
+        WritableUtils.writeVInt(output, storeColsInSingleCell ? 1 : -1);
     }
 
     public int getEstimatedByteSize() {
@@ -1241,16 +1423,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
      * Init calculated state reading/creating
      */
     private void initCachedState() {
-        dataEmptyKeyValueRef =
-                new ColumnReference(emptyKeyValueCFPtr.copyBytesIfNecessary(),
-                        QueryConstants.EMPTY_COLUMN_BYTES);
-
-        indexQualifiers = Lists.newArrayListWithExpectedSize(this.coveredColumns.size());
-        for (ColumnReference ref : coveredColumns) {
-            indexQualifiers.add(new ImmutableBytesPtr(IndexUtil.getIndexColumnName(
-                ref.getFamily(), ref.getQualifier())));
-        }
-
+        byte[] emptyKvQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(usesEncodedColumnNames).getFirst();
+        dataEmptyKeyValueRef = new ColumnReference(emptyKeyValueCFPtr.copyBytesIfNecessary(), emptyKvQualifier);
+        emptyKeyValueQualifierPtr = new ImmutableBytesPtr(emptyKvQualifier);
         this.allColumns = Sets.newLinkedHashSetWithExpectedSize(indexedExpressions.size() + coveredColumns.size());
         // columns that are required to evaluate all expressions in indexedExpressions (not including columns in data row key)
         this.indexedColumns = Sets.newLinkedHashSetWithExpectedSize(indexedExpressions.size());
@@ -1258,7 +1433,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         	KeyValueExpressionVisitor visitor = new KeyValueExpressionVisitor() {
                 @Override
                 public Void visit(KeyValueColumnExpression expression) {
-                	if (indexedColumns.add(new ColumnReference(expression.getColumnFamily(), expression.getColumnName()))) {
+                	if (indexedColumns.add(new ColumnReference(expression.getColumnFamily(), expression.getColumnQualifier()))) {
                 		indexedColumnTypes.add(expression.getDataType());
                 	}
                     return null;
@@ -1523,4 +1698,26 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             return udfParseNodes;
         }
     }
+    
+    public byte[] getEmptyKeyValueQualifier() {
+        return emptyKeyValueQualifierPtr.copyBytes();
+    }
+    
+    public Set<Pair<String, String>> getCoveredColumnInfo() {
+        return coveredColumnsInfo;
+    }
+    
+    public Set<Pair<String, String>> getIndexedColumnInfo() {
+        return indexedColumnsInfo;
+    }
+    
+    public StorageScheme getIndexStorageScheme() {
+        if (storeColsInSingleCell) {
+            return StorageScheme.COLUMNS_STORED_IN_SINGLE_CELL;
+        }
+        if (usesEncodedColumnNames) {
+            return StorageScheme.ENCODED_COLUMN_NAMES;
+        }
+        return StorageScheme.NON_ENCODED_COLUMN_NAMES;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
index bf1d0fb..05211c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexBuilder.java
@@ -166,7 +166,7 @@ public class PhoenixIndexBuilder extends NonTxIndexBuilder {
             ExpressionVisitor<Void> visitor = new StatelessTraverseAllExpressionVisitor<Void>() {
                 @Override
                 public Void visit(KeyValueColumnExpression expression) {
-                    get.addColumn(expression.getColumnFamily(), expression.getColumnName());
+                    get.addColumn(expression.getColumnFamily(), expression.getColumnQualifier());
                     estimatedSizeHolder[0]++;
                     return null;
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index eb73d6b..d382005 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -158,12 +158,13 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
                     .getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()));
             // Mimic the Put that gets generated by the client on an update of the index state
             Put put = new Put(indexTableKey);
-            if (blockWriteRebuildIndex) 
+            if (blockWriteRebuildIndex) {
                 put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
                         PIndexState.ACTIVE.getSerializedBytes());
-            else  
+            } else {  
                 put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_STATE_BYTES,
                         PIndexState.DISABLE.getSerializedBytes());
+            }
             put.add(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES, PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES,
                 PLong.INSTANCE.toBytes(minTimeStamp));
             final List<Mutation> tableMetadata = Collections.<Mutation>singletonList(put);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
index c67da6e..9ee5ea7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixTransactionalIndexer.java
@@ -67,7 +67,6 @@ import org.apache.phoenix.hbase.index.covered.update.IndexedColumnGroup;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.write.IndexWriter;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.trace.TracingUtils;
 import org.apache.phoenix.trace.util.NullSpan;
@@ -304,8 +303,16 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
                 for (ColumnReference ref : mutableColumns) {
                     scan.addColumn(ref.getFamily(), ref.getQualifier());
                 }
+                /*
+                 * Indexes inherit the storage scheme of the data table which means all the indexes have the same
+                 * storage scheme and empty key value qualifier. Note that this assumption would be broken if we start
+                 * supporting new indexes over existing data tables to have a different storage scheme than the data
+                 * table.
+                 */
+                byte[] emptyKeyValueQualifier = indexMaintainers.get(0).getEmptyKeyValueQualifier();
+                
                 // Project empty key value column
-                scan.addColumn(indexMaintainers.get(0).getDataEmptyKeyValueCF(), QueryConstants.EMPTY_COLUMN_BYTES);
+                scan.addColumn(indexMaintainers.get(0).getDataEmptyKeyValueCF(), emptyKeyValueQualifier);
                 ScanRanges scanRanges = ScanRanges.create(SchemaUtil.VAR_BINARY_SCHEMA, Collections.singletonList(keys), ScanUtil.SINGLE_COLUMN_SLOT_SPAN, KeyRange.EVERYTHING_RANGE, null, true, -1);
                 scanRanges.initializeScan(scan);
                 TableName tableName = env.getRegion().getRegionInfo().getTable();
@@ -356,7 +363,8 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
             Map<ImmutableBytesPtr, MultiMutation> mutationsToFindPreviousValue) throws IOException {
         if (scanner != null) {
             Result result;
-            ColumnReference emptyColRef = new ColumnReference(indexMetaData.getIndexMaintainers().get(0).getDataEmptyKeyValueCF(), QueryConstants.EMPTY_COLUMN_BYTES);
+            ColumnReference emptyColRef = new ColumnReference(indexMetaData.getIndexMaintainers().get(0)
+                    .getDataEmptyKeyValueCF(), indexMetaData.getIndexMaintainers().get(0).getEmptyKeyValueQualifier());
             // Process existing data table rows by removing the old index row and adding the new index row
             while ((result = scanner.next()) != null) {
                 Mutation m = mutationsToFindPreviousValue.remove(new ImmutableBytesPtr(result.getRow()));
@@ -384,7 +392,7 @@ public class PhoenixTransactionalIndexer extends BaseRegionObserver {
             // to generate point delete markers for all index rows that were added. We don't have Tephra
             // manage index rows in change sets because we don't want to be hit with the additional
             // memory hit and do not need to do conflict detection on index rows.
-            ColumnReference emptyColRef = new ColumnReference(indexMetaData.getIndexMaintainers().get(0).getDataEmptyKeyValueCF(), QueryConstants.EMPTY_COLUMN_BYTES);
+            ColumnReference emptyColRef = new ColumnReference(indexMetaData.getIndexMaintainers().get(0).getDataEmptyKeyValueCF(), indexMetaData.getIndexMaintainers().get(0).getEmptyKeyValueQualifier());
             while ((result = scanner.next()) != null) {
                 Mutation m = mutations.remove(new ImmutableBytesPtr(result.getRow()));
                 // Sort by timestamp, type, cf, cq so we can process in time batches from oldest to newest

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 25f3bec..225ae99 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
@@ -17,12 +17,18 @@
  */
 package org.apache.phoenix.iterate;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_ACTUAL_START_ROW;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_START_ROW_SUFFIX;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_STOP_ROW_SUFFIX;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QUERY_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_EMPTY_COLUMN_NAME;
+import static org.apache.phoenix.schema.PTable.IndexType.LOCAL;
+import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
+import static org.apache.phoenix.util.EncodedColumnsUtil.getEncodedColumnQualifier;
+import static org.apache.phoenix.util.ScanUtil.setQualifierRanges;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
@@ -76,20 +82,25 @@ import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.StorageScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
+import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
 import org.apache.phoenix.schema.stats.StatisticsUtil;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.LogUtil;
 import org.apache.phoenix.util.PrefixByteCodec;
 import org.apache.phoenix.util.PrefixByteDecoder;
@@ -157,7 +168,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         return true;
     }
     
-    private static void initializeScan(QueryPlan plan, Integer perScanLimit, Integer offset, Scan scan) {
+    private static void initializeScan(QueryPlan plan, Integer perScanLimit, Integer offset, Scan scan) throws SQLException {
         StatementContext context = plan.getContext();
         TableRef tableRef = plan.getTableRef();
         PTable table = tableRef.getTable();
@@ -208,7 +219,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                             // Project empty key value unless the column family containing it has
                             // been projected in its entirety.
                             if (!familyMap.containsKey(ecf) || familyMap.get(ecf) != null) {
-                                scan.addColumn(ecf, QueryConstants.EMPTY_COLUMN_BYTES);
+                                scan.addColumn(ecf, EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst());
                             }
                         }
                     }
@@ -226,7 +237,6 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             if(offset!=null){
                 ScanUtil.addOffsetAttribute(scan, offset);
             }
-
             int cols = plan.getGroupBy().getOrderPreservingColumnCount();
             if (cols > 0 && keyOnlyFilter &&
                 !plan.getStatement().getHint().hasHint(HintNode.Hint.RANGE_SCAN) &&
@@ -238,13 +248,86 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                         new DistinctPrefixFilter(plan.getTableRef().getTable().getRowKeySchema(),
                                 cols));
             }
-
+            // When analyzing the table, there is no look up for key values being done.
+            // So there is no point setting the range.
+            if (setQualifierRanges(table) && !ScanUtil.isAnalyzeTable(scan)) {
+                Pair<Integer, Integer> range = getEncodedQualifierRange(scan, context);
+                if (range != null) {
+                    scan.setAttribute(BaseScannerRegionObserver.MIN_QUALIFIER, getEncodedColumnQualifier(range.getFirst()));
+                    scan.setAttribute(BaseScannerRegionObserver.MAX_QUALIFIER, getEncodedColumnQualifier(range.getSecond()));
+                }
+            }
             if (optimizeProjection) {
                 optimizeProjection(context, scan, table, statement);
             }
         }
     }
+    
+    private static Pair<Integer, Integer> getEncodedQualifierRange(Scan scan, StatementContext context)
+            throws SQLException {
+        PTable table = context.getCurrentTable().getTable();
+        StorageScheme storageScheme = table.getStorageScheme();
+        checkArgument(storageScheme == StorageScheme.ENCODED_COLUMN_NAMES,
+            "Method should only be used for tables using encoded column names");
+        Pair<Integer, Integer> minMaxQualifiers = new Pair<>();
+        for (Pair<byte[], byte[]> whereCol : context.getWhereConditionColumns()) {
+            byte[] cq = whereCol.getSecond();
+            if (cq != null) {
+                int qualifier = getEncodedColumnQualifier(cq);
+                determineQualifierRange(qualifier, minMaxQualifiers);
+            }
+        }
+        Map<byte[], NavigableSet<byte[]>> familyMap = scan.getFamilyMap();
+
+        Map<String, Pair<Integer, Integer>> qualifierRanges = SchemaUtil.getQualifierRanges(table);
+        for (Entry<byte[], NavigableSet<byte[]>> entry : familyMap.entrySet()) {
+            if (entry.getValue() != null) {
+                for (byte[] cq : entry.getValue()) {
+                    if (cq != null) {
+                        int qualifier = getEncodedColumnQualifier(cq);
+                        determineQualifierRange(qualifier, minMaxQualifiers);
+                    }
+                }
+            } else {
+                /*
+                 * All the columns of the column family are being projected. So we will need to
+                 * consider all the columns in the column family to determine the min-max range.
+                 */
+                String family = Bytes.toString(entry.getKey());
+                if (table.getType() == INDEX && table.getIndexType() == LOCAL && !IndexUtil.isLocalIndexFamily(family)) {
+                    //TODO: samarth confirm with James why do we need this hack here :(
+                    family = IndexUtil.getLocalIndexColumnFamily(family);
+                }
+                Pair<Integer, Integer> range = qualifierRanges.get(family);
+                determineQualifierRange(range.getFirst(), minMaxQualifiers);
+                determineQualifierRange(range.getSecond(), minMaxQualifiers);
+            }
+        }
+        if (minMaxQualifiers.getFirst() == null) {
+            return null;
+        }
+        return minMaxQualifiers;
+    }
 
+    /**
+     * 
+     * @param cq
+     * @param minMaxQualifiers
+     * @return true if the empty column was projected
+     */
+    private static void determineQualifierRange(Integer qualifier, Pair<Integer, Integer> minMaxQualifiers) {
+        if (minMaxQualifiers.getFirst() == null) {
+            minMaxQualifiers.setFirst(qualifier);
+            minMaxQualifiers.setSecond(qualifier);
+        } else {
+            if (minMaxQualifiers.getFirst() > qualifier) {
+                minMaxQualifiers.setFirst(qualifier);
+            } else if (minMaxQualifiers.getSecond() < qualifier) {
+                minMaxQualifiers.setSecond(qualifier);
+            }
+        }
+    }
+    
     private static void optimizeProjection(StatementContext context, Scan scan, PTable table, FilterableStatement statement) {
         Map<byte[], NavigableSet<byte[]>> familyMap = scan.getFamilyMap();
         // columnsTracker contain cf -> qualifiers which should get returned.
@@ -341,7 +424,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             // the ExplicitColumnTracker not to be used, though.
             if (!statement.isAggregate() && filteredColumnNotInProjection) {
                 ScanUtil.andFilterAtEnd(scan, new ColumnProjectionFilter(SchemaUtil.getEmptyColumnFamily(table),
-                        columnsTracker, conditionOnlyCfs));
+                        columnsTracker, conditionOnlyCfs, EncodedColumnsUtil.usesEncodedColumnNames(table.getStorageScheme())));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/iterate/LookAheadResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/LookAheadResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LookAheadResultIterator.java
index 3293f65..1e5f09e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/LookAheadResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/LookAheadResultIterator.java
@@ -49,7 +49,7 @@ abstract public class LookAheadResultIterator implements PeekingResultIterator {
         };
     }
     
-    private final static Tuple UNINITIALIZED = new ResultTuple();
+    private final static Tuple UNINITIALIZED = ResultTuple.EMPTY_TUPLE;
     private Tuple next = UNINITIALIZED;
     
     abstract protected Tuple advance() throws SQLException;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferQueue.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferQueue.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferQueue.java
index 8ada952..135ab26 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferQueue.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/MappedByteBufferQueue.java
@@ -180,6 +180,7 @@ public abstract class MappedByteBufferQueue<T> extends AbstractQueue<T> {
             return this.index;
         }
         
+        @Override
         public int size() {
             if (flushBuffer)
                 return flushedCount;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
index 8dcb2e8..e4c52c0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/OrderedResultIterator.java
@@ -32,6 +32,7 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.OrderByExpression;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.SizedUtil;
 
 import com.google.common.base.Function;
@@ -264,7 +265,7 @@ public class OrderedResultIterator implements PeekingResultIterator {
             }
             this.byteSize = queueEntries.getByteSize();
         } catch (IOException e) {
-            throw new SQLException("", e);
+            ServerUtil.createIOException(e.getMessage(), e);
         } finally {
             delegate.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
index 88e141a..816b78c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/iterate/RegionScannerResultIterator.java
@@ -24,16 +24,24 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.schema.tuple.EncodedColumnQualiferCellsList;
 import org.apache.phoenix.schema.tuple.MultiKeyValueTuple;
+import org.apache.phoenix.schema.tuple.PositionBasedMultiKeyValueTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
 
 
 public class RegionScannerResultIterator extends BaseResultIterator {
     private final RegionScanner scanner;
+    private final Pair<Integer, Integer> minMaxQualifiers;
+    private final boolean useQualifierAsIndex;
     
-    public RegionScannerResultIterator(RegionScanner scanner) {
+    public RegionScannerResultIterator(RegionScanner scanner, Pair<Integer, Integer> minMaxQualifiers) {
         this.scanner = scanner;
+        this.useQualifierAsIndex = ScanUtil.useQualifierAsIndex(minMaxQualifiers);
+        this.minMaxQualifiers = minMaxQualifiers;
     }
     
     @Override
@@ -43,7 +51,7 @@ public class RegionScannerResultIterator extends BaseResultIterator {
         synchronized (scanner) {
             try {
                 // TODO: size
-                List<Cell> results = new ArrayList<Cell>();
+                List<Cell> results = useQualifierAsIndex ? new EncodedColumnQualiferCellsList(minMaxQualifiers.getFirst(), minMaxQualifiers.getSecond()) :  new ArrayList<Cell>();
                 // Results are potentially returned even when the return value of s.next is false
                 // since this is an indication of whether or not there are more values after the
                 // ones returned
@@ -53,7 +61,7 @@ public class RegionScannerResultIterator extends BaseResultIterator {
                 }
                 // We instantiate a new tuple because in all cases currently we hang on to it
                 // (i.e. to compute and hold onto the TopN).
-                MultiKeyValueTuple tuple = new MultiKeyValueTuple();
+                Tuple tuple = useQualifierAsIndex ? new PositionBasedMultiKeyValueTuple() : new MultiKeyValueTuple();
                 tuple.setKeyValues(results);
                 return tuple;
             } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/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 9c5d521..d312580 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
@@ -205,7 +205,7 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     public static final byte[] BASE_COLUMN_COUNT_BYTES = Bytes.toBytes(BASE_COLUMN_COUNT);
     public static final String IS_ROW_TIMESTAMP = "IS_ROW_TIMESTAMP";
     public static final byte[] IS_ROW_TIMESTAMP_BYTES = Bytes.toBytes(IS_ROW_TIMESTAMP);
-
+    
     public static final String TABLE_FAMILY = QueryConstants.DEFAULT_COLUMN_FAMILY;
     public static final byte[] TABLE_FAMILY_BYTES = QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES;
 
@@ -320,6 +320,13 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
     /** Version below which we fall back on the generic KeyValueBuilder */
     public static final int CLIENT_KEY_VALUE_BUILDER_THRESHOLD = VersionUtil.encodeVersion("0", "94", "14");
     
+    public static final String STORAGE_SCHEME = "STORAGE_SCHEME";
+    public static final byte[] STORAGE_SCHEME_BYTES = Bytes.toBytes(STORAGE_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";
+    public static final byte[] COLUMN_QUALIFIER_COUNTER_BYTES = Bytes.toBytes(COLUMN_QUALIFIER_COUNTER);
+
     PhoenixDatabaseMetaData(PhoenixConnection connection) throws SQLException {
         this.emptyResultSet = new PhoenixResultSet(ResultIterator.EMPTY_ITERATOR, RowProjector.EMPTY_PROJECTOR, new StatementContext(new PhoenixStatement(connection), false));
         this.connection = connection;
@@ -593,9 +600,8 @@ public class PhoenixDatabaseMetaData implements DatabaseMetaData {
                 newCells.addAll(cells);
                 newCells.add(kv);
                 Collections.sort(newCells, KeyValue.COMPARATOR);
-                resultTuple.setResult(Result.create(newCells));
+                tuple = new ResultTuple(Result.create(newCells));
             }
-
             return tuple;
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
index 47c17ae..3ca48a1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/jdbc/PhoenixResultSet.java
@@ -107,7 +107,7 @@ public class PhoenixResultSet implements ResultSet, SQLCloseable {
     private final static String STRING_FALSE = "0";
     private final static BigDecimal BIG_DECIMAL_FALSE = BigDecimal.valueOf(0);
     private final static Integer INTEGER_FALSE = Integer.valueOf(0);
-    private final static Tuple BEFORE_FIRST = new ResultTuple();
+    private final static Tuple BEFORE_FIRST = ResultTuple.EMPTY_TUPLE;
 
     private final ResultIterator scanner;
     private final RowProjector rowProjector;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
index 908a117..2d7550a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/join/HashCacheFactory.java
@@ -122,6 +122,7 @@ public class HashCacheFactory implements ServerCacheFactory {
                     int resultSize = (int)Bytes.readVLong(hashCacheByteArray, offset);
                     offset += WritableUtils.decodeVIntSize(hashCacheByteArray[offset]);
                     ImmutableBytesWritable value = new ImmutableBytesWritable(hashCacheByteArray,offset,resultSize);
+                    //TODO: samarth make joins work with position look up.
                     Tuple result = new ResultTuple(ResultUtil.toResult(value));
                     ImmutableBytesPtr key = TupleUtil.getConcatenatedValue(result, onExpressions);
                     List<Tuple> tuples = hashCacheMap.get(key);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
index b12326a..a6a57c7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/FormatToBytesWritableMapper.java
@@ -49,6 +49,7 @@ import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.util.ColumnInfo;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -208,7 +209,7 @@ public abstract class FormatToBytesWritableMapper<RECORD> extends Mapper<LongWri
     not care about it
      */
     private void initColumnIndexes() throws SQLException {
-        columnIndexes = new TreeMap(Bytes.BYTES_COMPARATOR);
+        columnIndexes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
         int columnIndex = 0;
         for(int index = 0; index < logicalNames.size(); index++) {
             PTable table = PhoenixRuntime.getTable(conn, logicalNames.get(index));
@@ -216,18 +217,23 @@ public abstract class FormatToBytesWritableMapper<RECORD> extends Mapper<LongWri
             for (int i = 0; i < cls.size(); i++) {
                 PColumn c = cls.get(i);
                 byte[] family = new byte[0];
-                if (c.getFamilyName() != null)  // Skip PK column
+                byte[] cq;
+                if (!SchemaUtil.isPKColumn(c)) {
                     family = c.getFamilyName().getBytes();
-                byte[] name = c.getName().getBytes();
-                byte[] cfn = Bytes.add(family, QueryConstants.NAMESPACE_SEPARATOR_BYTES, name);
+                    cq = EncodedColumnsUtil.getColumnQualifier(c, table);
+                } else {
+                    // TODO: samarth verify if this is the right thing to do here.
+                    cq = c.getName().getBytes();
+                }
+                byte[] cfn = Bytes.add(family, QueryConstants.NAMESPACE_SEPARATOR_BYTES, cq);
                 if (!columnIndexes.containsKey(cfn)) {
                     columnIndexes.put(cfn, new Integer(columnIndex));
                     columnIndex++;
                 }
             }
             byte[] emptyColumnFamily = SchemaUtil.getEmptyColumnFamily(table);
-            byte[] cfn = Bytes.add(emptyColumnFamily, QueryConstants.NAMESPACE_SEPARATOR_BYTES,
-                    QueryConstants.EMPTY_COLUMN_BYTES);
+            byte[] emptyKeyValue = EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
+            byte[] cfn = Bytes.add(emptyColumnFamily, QueryConstants.NAMESPACE_SEPARATOR_BYTES, emptyKeyValue);
             columnIndexes.put(cfn, new Integer(columnIndex));
             columnIndex++;
         }
@@ -243,9 +249,9 @@ public abstract class FormatToBytesWritableMapper<RECORD> extends Mapper<LongWri
     private int findIndex(Cell cell) throws IOException {
         byte[] familyName = Bytes.copy(cell.getFamilyArray(), cell.getFamilyOffset(),
                 cell.getFamilyLength());
-        byte[] name = Bytes.copy(cell.getQualifierArray(), cell.getQualifierOffset(),
+        byte[] cq = Bytes.copy(cell.getQualifierArray(), cell.getQualifierOffset(),
                 cell.getQualifierLength());
-        byte[] cfn = Bytes.add(familyName, QueryConstants.NAMESPACE_SEPARATOR_BYTES, name);
+        byte[] cfn = Bytes.add(familyName, QueryConstants.NAMESPACE_SEPARATOR_BYTES, cq);
         if(columnIndexes.containsKey(cfn)) {
             return columnIndexes.get(cfn);
         }


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

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

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

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

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

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

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

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

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

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

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

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


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

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


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

Branch: refs/heads/encodecolumns2
Commit: 56c17679adee1837ea1e2504eca0d2ebc1c5b685
Parents: d033aba
Author: Samarth <sa...@salesforce.com>
Authored: Wed Oct 5 00:11:07 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Mon Nov 7 23:54:24 2016 -0800

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


http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index adadca7..8275f3f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.end2end;
 
+import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
 import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.junit.Assert.assertEquals;
@@ -476,14 +477,14 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             // For a diverged view, only base table's pk column will be added and that too at the end.
             assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
             
-            // Add existing column VIEW_COL2 to the base table
+            // Adding existing column VIEW_COL2 to the base table isn't allowed.
             alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
-            conn.createStatement().execute(alterBaseTable);
-            
-            // For the non-diverged view, adding the column VIEW_COL2 will end up changing its ordinal position in the view.
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 2, 9, 8, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL2", "VIEW_COL1");
-            // For the diverged view, adding the column VIEW_COL2 will not change its ordinal position in the view. It also won't change the base column count or the sequence number
-            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
+            try {
+                conn.createStatement().execute(alterBaseTable);
+                fail();
+            } catch (SQLException e) {
+                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }
         }
     }
     
@@ -500,13 +501,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the both columns were added to view1
@@ -530,13 +531,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the dropped columns aren't visible
@@ -569,7 +570,7 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 String view1DDL = "CREATE VIEW " + view1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 CHAR(256)) AS SELECT * FROM " + baseTable;
                 tenant1Conn.createStatement().execute(view1DDL);
                 // This should not modify the base table
-                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR";
+                String alterView = "ALTER VIEW " + view1 + " ADD NEWCOL1 VARCHAR, NEWCOL5 VARCHAR";
                 tenant1Conn.createStatement().execute(alterView);
                 HTableDescriptor tableDesc2 = tenant1Conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin().getTableDescriptor(Bytes.toBytes(baseTable));
                 assertEquals(tableDesc1, tableDesc2);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 48f4217..276390a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -18,6 +18,15 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.hadoop.hbase.HColumnDescriptor.DEFAULT_REPLICATION_SCOPE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_FAMILY;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODED_COLUMN_QUALIFIER;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM;
+import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.closeConnection;
 import static org.apache.phoenix.util.TestUtil.closeStatement;
@@ -53,6 +62,7 @@ import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.EncodedCQCounter;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.IndexUtil;
@@ -231,8 +241,18 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals(exists, rs.next());
     }
 
+    
+    @Test
+    public void testDropIndexedColumnImmutableIndex() throws Exception {
+        helpTestDropIndexedColumn(true);
+    }
+    
     @Test
-    public void testDropIndexedColumn() throws Exception {
+    public void testDropIndexedColumnMutableIndex() throws Exception {
+        helpTestDropIndexedColumn(false);
+    }
+    
+    private void helpTestDropIndexedColumn(boolean immutable) throws Exception {
         String query;
         ResultSet rs;
         PreparedStatement stmt;
@@ -244,7 +264,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         // make sure that the tables are empty, but reachable
         conn.createStatement().execute(
           "CREATE TABLE " + dataTableFullName
-              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + (immutable ? "IMMUTABLE_ROWS = true" : ""));
         query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
@@ -2118,8 +2138,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class); 
             PTable table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName));
             // Assert that the column shows up as row time stamp in the cache.
-            assertTrue(table.getColumn("PK1").isRowTimestamp());
-            assertFalse(table.getColumn("PK2").isRowTimestamp());
+            assertTrue(table.getPColumnForColumnName("PK1").isRowTimestamp());
+            assertFalse(table.getPColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName, "PK1");
             
             String dataTableName2 = BaseTest.generateUniqueName();
@@ -2127,18 +2147,17 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute("CREATE TABLE " + dataTableFullName2 + " (PK1 VARCHAR, PK2 DATE PRIMARY KEY ROW_TIMESTAMP, KV1 VARCHAR, KV2 INTEGER)");
             table = phxConn.getTable(new PTableKey(phxConn.getTenantId(), dataTableFullName2));
             // Assert that the column shows up as row time stamp in the cache.
-            assertFalse(table.getColumn("PK1").isRowTimestamp());
-            assertTrue(table.getColumn("PK2").isRowTimestamp());
+            assertFalse(table.getPColumnForColumnName("PK1").isRowTimestamp());
+            assertTrue(table.getPColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName2, "PK2");
             
             // Create an index on a table has a row time stamp pk column. The column should show up as a row time stamp column for the index too. 
             conn.createStatement().execute("CREATE INDEX " + indexTableName + "  ON " + dataTableFullName2 + " (KV1) include (KV2)");
             PTable indexTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), indexTableFullName));
-            String indexColName = IndexUtil.getIndexColumnName(table.getColumn("PK2"));
+            String indexColName = IndexUtil.getIndexColumnName(table.getPColumnForColumnName("PK2"));
             // Assert that the column shows up as row time stamp in the cache.
-            assertTrue(indexTable.getColumn(indexColName).isRowTimestamp());
+            assertTrue(indexTable.getPColumnForColumnName(indexColName).isRowTimestamp());
             assertIsRowTimestampSet(schemaName, indexTableName, indexColName);
-            
             String viewTableName2 = dataTableName2 + "_VIEW";
             String viewTableFullName2 = SchemaUtil.getTableName(schemaName, viewTableName2);
             // Creating a view with a row_timestamp column in its pk constraint is not allowed
@@ -2207,5 +2226,254 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 		}
 	}
 	
+	@Test
+	public void testMetadataForImmutableTableWithEncodedColumns() throws Exception {
+	    String schemaName = "XYZ";
+	    String baseTableName = generateUniqueName();
+	    String viewName = generateUniqueName();
+	    String fullTableName = schemaName + "." + baseTableName;
+	    String fullViewName = schemaName + "." + viewName;
+	    try (Connection conn = DriverManager.getConnection(getUrl())) {
+	        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+	        conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + fullTableName + " ("
+	                + " ID char(1) NOT NULL,"
+	                + " COL1 integer NOT NULL,"
+	                + " COL2 bigint NOT NULL,"
+	                + " KV1 VARCHAR"
+	                + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+	                + " )  IMMUTABLE_ROWS = true");
+	        PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+	        long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
+
+	        // assert that the client side cache is updated.
+	        EncodedCQCounter cqCounter = baseTable.getEncodedCQCounter();
+	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
+	        
+	        
+	        // assert that the server side metadata is updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "KV1", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber);
+
+	        // now create a view and validate client and server side metadata
+	        String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
+	        conn.createStatement().execute(viewDDL);
+	        baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+	        PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+
+	        // verify that the client side cache is updated. Base table's cq counters should be updated.
+	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2), baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), baseTable.getEncodedCQCounter().getNextQualifier("A"));
+	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        
+	        // assert that the server side metadata for the base table and the view is also updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2, true);
+	        assertEncodedCQCounter("A", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+	        assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 1);
+	        assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM);
+	    }
+	}
+	
+	@Test
+	public void testMetadataForMutableTableWithEncodedColumns() throws Exception {
+	    String schemaName = "XYZ";
+	    String baseTableName = generateUniqueName();
+	    String viewName = generateUniqueName();
+	    String fullTableName = schemaName + "." + baseTableName;
+	    String fullViewName = schemaName + "." + viewName;
+	    try (Connection conn = DriverManager.getConnection(getUrl())) {
+	        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+	        conn.createStatement().execute("CREATE TABLE IF NOT EXISTS " + fullTableName + " ("
+	                + " ID char(1) NOT NULL,"
+	                + " COL1 integer NOT NULL,"
+	                + " COL2 bigint NOT NULL,"
+	                + " KV1 VARCHAR"
+	                + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+	                + " )");
+	        PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+	        long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
+
+	        // assert that the client side cache is updated.
+	        EncodedCQCounter cqCounter = baseTable.getEncodedCQCounter();
+	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
+
+
+	        // assert that the server side metadata is updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "KV1", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber);
+
+	        // now create a view and validate client and server side metadata
+	        String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
+	        conn.createStatement().execute(viewDDL);
+	        baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+	        PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+
+	        // verify that the client side cache is updated. Base table's cq counters should be updated.
+	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3), baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+
+	        // assert that the server side metadata for the base table and the view is also updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3, true);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+	        assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 1);
+	        assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM);
+	    }
+	}
+	
+	@Test
+    public void testAddingColumnsToTablesAndViewsWithEncodedColumns() throws Exception {
+        String schemaName = "XYZ";
+        String baseTableName = generateUniqueName();
+        String viewName = generateUniqueName();
+        String fullTableName = schemaName + "." + baseTableName;
+        String fullViewName = schemaName + "." + viewName;
+        Properties props = new Properties();
+        props.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(true));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute("CREATE SCHEMA " + schemaName);
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            conn.createStatement().execute("CREATE TABLE " + fullTableName + " ("
+                    + " ID char(1) NOT NULL,"
+                    + " COL1 integer NOT NULL,"
+                    + " COL2 bigint NOT NULL,"
+                    + " CONSTRAINT NAME_PK PRIMARY KEY (ID, COL1, COL2)"
+                    + " )");
+            PTable baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+            long initBaseTableSeqNumber = baseTable.getSequenceNumber(); 
+
+            // Add a column to the base table and see if the client and server metadata is updated correctly
+            String alterDDL = "ALTER TABLE " + fullTableName + " ADD COL3 VARCHAR PRIMARY KEY, COL4 INTEGER, COL5 VARCHAR, B.COL6 DECIMAL (10, 2)";
+            conn.createStatement().execute(alterDDL);
+
+            // assert that the client side cache is updated.
+            baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+            EncodedCQCounter encodedCqCounter = baseTable.getEncodedCQCounter();
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3), encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert that the server side metadata is updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3, true);
+            
+            // assert that the server side metadata for columns is updated correctly.
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL4", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL5", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+            assertEncodedCQValue("B", "COL6", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 1);
+
+            // Create a view
+            String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
+            conn.createStatement().execute(viewDDL);
+            
+            // assert that the server side metadata is updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5, true);
+            
+            // assert that the server side metadata for columns is updated correctly.
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+            assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 4);
+            // Creating a view that adds its own columns should increment the base table's sequence number too.
+            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 2);
+            
+
+            // Add column to the view
+            viewDDL = "ALTER VIEW " + fullViewName + " ADD VIEW_COL3 DECIMAL(10, 2), A.VIEW_COL4 VARCHAR, B.VIEW_COL5 INTEGER";
+            conn.createStatement().execute(viewDDL);
+
+            // assert that the client cache for the base table is updated
+            baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+            encodedCqCounter = baseTable.getEncodedCQCounter();
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 8), encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert client cache for view
+            PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+            encodedCqCounter = view.getEncodedCQCounter();
+            assertNull("A view should always have the column qualifier counter as null", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert that the server side metadata for the base table and the view is also updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 8, true);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+            assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 4);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL3", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5);
+            assertEncodedCQValue("A", "VIEW_COL4", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 6);
+            assertEncodedCQValue("B", "VIEW_COL5", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 7);
+            // Adding a column to the should increment the base table's sequence number too since we update the cq counters for column families.
+            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 3);
+            assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 1);
+            
+            // Add column to the base table which doesn't already exist in the view.
+            alterDDL = "ALTER TABLE " + fullTableName + " ADD COL10 VARCHAR, A.COL11 INTEGER";
+            conn.createStatement().execute(alterDDL);
+            baseTable = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullTableName));
+            
+            // assert that the client cache for the base table is updated 
+            encodedCqCounter = baseTable.getEncodedCQCounter();
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 10), encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert client cache for view
+            view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+            encodedCqCounter = view.getEncodedCQCounter();
+            assertNull("A view should always have the column qualifier counter as null", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert that the server side metadata for the base table and the view is also updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 10), true);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL10", schemaName, viewName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 8));
+            assertEncodedCQValue("A", "COL11", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 9);
+            assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + 4);
+            assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM + 2);
+        }
+    }
+	
+	private void assertEncodedCQValue(String columnFamily, String columnName, String schemaName, String tableName, int expectedValue) throws Exception {
+        String query = "SELECT " + ENCODED_COLUMN_QUALIFIER + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
+                + " = ? " + " AND " + COLUMN_FAMILY + " = ?" + " AND " + COLUMN_NAME  + " = ?";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, schemaName);
+            stmt.setString(2, tableName);
+            stmt.setString(3, columnFamily);
+            stmt.setString(4, columnName);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(expectedValue, rs.getInt(1));
+            assertFalse(rs.next());
+        }
+    }
+    
+    private void assertEncodedCQCounter(String columnFamily, String schemaName, String tableName, int expectedValue, boolean rowExists) throws Exception {
+        String query = "SELECT " + COLUMN_QUALIFIER_COUNTER + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
+                + " = ? " + " AND " + COLUMN_FAMILY + " = ? AND " + COLUMN_QUALIFIER_COUNTER + " IS NOT NULL";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, schemaName);
+            stmt.setString(2, tableName);
+            stmt.setString(3, columnFamily);
+            ResultSet rs = stmt.executeQuery();
+            if (rowExists) {
+                assertTrue(rs.next());
+                assertEquals(expectedValue, rs.getInt(1));
+                assertFalse(rs.next());
+            } else {
+                assertFalse(rs.next());
+            }
+        }
+    }
+    
+    private void assertSequenceNumber(String schemaName, String tableName, long expectedSequenceNumber) throws Exception {
+        String query = "SELECT " + TABLE_SEQ_NUM + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
+                + " = ? AND " +  TABLE_SEQ_NUM + " IS NOT NULL AND " + COLUMN_NAME + " IS NULL AND "
+                + COLUMN_FAMILY + " IS NULL ";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            PreparedStatement stmt = conn.prepareStatement(query);
+            stmt.setString(1, schemaName);
+            stmt.setString(2, tableName);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            assertEquals(expectedSequenceNumber, rs.getInt(1));
+            assertFalse(rs.next());
+        }
+    }
+	
 }
  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/56c17679/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
index e6bf2d2..d0bc2c0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableWithViewsIT.java
@@ -65,7 +65,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     public static Collection<Boolean> data() {
         return Arrays.asList(false, true);
     }
-	
+    
     private String generateDDL(String format) {
         return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
             isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");
@@ -91,7 +91,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             // adding a new pk column and a new regular column
             conn.createStatement().execute("ALTER TABLE " + tableName + " ADD COL3 varchar(10) PRIMARY KEY, COL4 integer");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 7, 5, "ID", "COL1", "COL2", "COL3", "COL4", "VIEW_COL1", "VIEW_COL2");
         } 
     }
@@ -119,7 +119,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
 
             // drop two columns from the base table
             conn.createStatement().execute("ALTER TABLE " + tableName + " DROP COLUMN COL3, COL5");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 4,
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 4,
                 QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL4");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 6, 4,
                 "ID", "COL1", "COL2", "COL4", "VIEW_COL1", "VIEW_COL2");
@@ -163,73 +163,80 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             viewConn.commit();
             
             try {
-                // should fail because there is already a view column with same name of different type
-                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 char(10)");
-                fail();
-            }
-            catch (SQLException e) {
-                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-            }           
-            
-            try {
-                // should fail because there is already a view column with same name with different scale
-                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,1)");
-                fail();
-            }
-            catch (SQLException e) {
-                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-            } 
-            
-            try {
-                // should fail because there is already a view column with same name with different length
-                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(9,2)");
+                // adding a key value column to the base table that already exists in the view is not allowed
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
                 fail();
-            }
-            catch (SQLException e) {
-                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
-            } 
-            
-            try {
-                // should fail because there is already a view column with different length
-                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR");
-                fail();
-            }
-            catch (SQLException e) {
+            } catch (SQLException e) {
                 assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
             }
-            
-            // validate that there were no columns added to the table or view
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
-            
-            // should succeed 
-            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
-            
-            // query table
-            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
-            assertTrue(rs.next());
-            assertEquals("view1", rs.getString("ID"));
-            assertEquals(12, rs.getInt("COL1"));
-            assertEquals(13, rs.getInt("COL2"));
-            assertEquals("view5", rs.getString("VIEW_COL2"));
-            assertEquals(17, rs.getInt("VIEW_COL4"));
-            assertFalse(rs.next());
-
-            // query view
-            rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
-            assertTrue(rs.next());
-            assertEquals("view1", rs.getString("ID"));
-            assertEquals(12, rs.getInt("COL1"));
-            assertEquals(13, rs.getInt("COL2"));
-            assertEquals(14, rs.getInt("VIEW_COL1"));
-            assertEquals("view5", rs.getString("VIEW_COL2"));
-            assertEquals("view6", rs.getString("VIEW_COL3"));
-            assertEquals(17, rs.getInt("VIEW_COL4"));
-            assertEquals(18, rs.getInt("VIEW_COL5"));
-            assertEquals("view9", rs.getString("VIEW_COL6"));
-            assertFalse(rs.next());
+//            try {
+//                // should fail because there is already a view column with same name of different type
+//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 char(10)");
+//                fail();
+//            }
+//            catch (SQLException e) {
+//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+//            }           
+//            
+//            try {
+//                // should fail because there is already a view column with same name with different scale
+//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(10,1)");
+//                fail();
+//            }
+//            catch (SQLException e) {
+//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+//            } 
+//            
+//            try {
+//                // should fail because there is already a view column with same name with different length
+//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL1 DECIMAL(9,2)");
+//                fail();
+//            }
+//            catch (SQLException e) {
+//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+//            } 
+//            
+//            try {
+//                // should fail because there is already a view column with different length
+//                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL2 VARCHAR");
+//                fail();
+//            }
+//            catch (SQLException e) {
+//                assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+//            }
+//            
+//            // validate that there were no columns added to the table or view
+//            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+//            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
+//            
+//            // should succeed 
+//            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+//            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 2, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
+//            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
+//            
+//            // query table
+//            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
+//            assertTrue(rs.next());
+//            assertEquals("view1", rs.getString("ID"));
+//            assertEquals(12, rs.getInt("COL1"));
+//            assertEquals(13, rs.getInt("COL2"));
+//            assertEquals("view5", rs.getString("VIEW_COL2"));
+//            assertEquals(17, rs.getInt("VIEW_COL4"));
+//            assertFalse(rs.next());
+//
+//            // query view
+//            rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
+//            assertTrue(rs.next());
+//            assertEquals("view1", rs.getString("ID"));
+//            assertEquals(12, rs.getInt("COL1"));
+//            assertEquals(13, rs.getInt("COL2"));
+//            assertEquals(14, rs.getInt("VIEW_COL1"));
+//            assertEquals("view5", rs.getString("VIEW_COL2"));
+//            assertEquals("view6", rs.getString("VIEW_COL3"));
+//            assertEquals(17, rs.getInt("VIEW_COL4"));
+//            assertEquals(18, rs.getInt("VIEW_COL5"));
+//            assertEquals("view9", rs.getString("VIEW_COL6"));
+//            assertFalse(rs.next());
         } 
     }
     
@@ -679,4 +686,4 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
         } 
     }
     
-}
+}
\ No newline at end of file

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

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

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

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

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

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

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

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

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

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

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

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

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

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