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/10/05 07:11:32 UTC

[1/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Repository: phoenix
Updated Branches:
  refs/heads/encodecolumns2 [created] 33ba1ee8e


http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..845b113 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,33 +17,44 @@
  */
 package org.apache.phoenix.schema.tuple;
 
+import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+
+import java.util.Collections;
+
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.GenericKeyValueBuilder;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PInteger;
 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()));
     
+    //TODO: samarth see if we can get rid of this constructor altogether.
     public ResultTuple(Result result) {
         this.result = result;
     }
     
-    public ResultTuple() {
-    }
+//    public ResultTuple(Result result, boolean useQualifierAsIndex) {
+//        this.result = result;
+//        this.useQualifierAsIndex = useQualifierAsIndex;
+//    }
     
     public Result getResult() {
         return this.result;
     }
 
-    public void setResult(Result result) {
-        this.result = result;
-    }
-    
     @Override
     public void getKey(ImmutableBytesWritable ptr) {
         ptr.set(result.getRow());
@@ -56,6 +67,12 @@ public class ResultTuple extends BaseTuple {
 
     @Override
     public KeyValue getValue(byte[] family, byte[] qualifier) {
+//        if (useQualifierAsIndex) {
+//            int index = PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, SortOrder.ASC);
+//            //TODO: samarth this seems like a hack here at this place. Think more. Maybe we should use a new tuple here?
+//            index = index >= ENCODED_CQ_COUNTER_INITIAL_VALUE ? (index - ENCODED_CQ_COUNTER_INITIAL_VALUE) : index;
+//            return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(result.rawCells()[index]);
+//        }
         Cell cell = KeyValueUtil.getColumnLatest(GenericKeyValueBuilder.INSTANCE, 
           result.rawCells(), family, qualifier);
         return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cell);
@@ -104,4 +121,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/33ba1ee8/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/33ba1ee8/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
new file mode 100644
index 0000000..02a85a5
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/UnboundedSkipNullCellsList.java
@@ -0,0 +1,488 @@
+/*
+ * 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.Arrays;
+import java.util.Collection;
+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.SortOrder;
+import org.apache.phoenix.schema.types.PInteger;
+
+import com.google.common.base.Preconditions;
+
+public class UnboundedSkipNullCellsList implements List<Cell> {
+    private int minQualifier;
+    private int maxQualifier;
+    private Cell[] array;
+    private int numNonNullElements;
+    private int firstNonNullElementIdx = -1;
+    private int leftBoundary;
+    private int rightBoundary;
+    
+    // extra capacity we have either at the start or at the end or at at both extremes
+    // to accommodate column qualifiers outside of the range (minQualifier, maxQualifier)
+    private static final int INIITAL_EXTRA_BUFFER = 10;  
+
+    public UnboundedSkipNullCellsList(int minQualifier, int maxQualifier) {
+        checkArgument(maxQualifier - minQualifier > 0, "Illegal arguments. MinQualifier: " + minQualifier + ". MaxQualifier: " + maxQualifier);
+        this.minQualifier = minQualifier;
+        this.maxQualifier = maxQualifier;
+        int minIndex = Math.max(0, minQualifier - INIITAL_EXTRA_BUFFER);
+        int maxIndex = maxQualifier + INIITAL_EXTRA_BUFFER;
+        int size = maxIndex - minIndex + 1;
+        this.array = new Cell[size];
+    }
+    
+    
+    @Override
+    public int size() {
+        return numNonNullElements;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return numNonNullElements == 0;
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return indexOf(o) >= 0;
+    }
+
+
+    /**
+     * This implementation only returns an array of non-null elements in the list.
+     */
+    @Override
+    public Object[] toArray() {
+        Object[] toReturn = new Object[numNonNullElements];
+        int counter = 0;
+        for (int i = 0; i < array.length; i++) {
+            if (array[i] != null) {
+                toReturn[counter++] = array[i];
+            }
+        }
+        return toReturn;
+    }
+
+
+    /**
+     * This implementation only returns an array of non-null elements in the list.
+     * This is not the most efficient way of copying elemts into an array 
+     */
+    @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 = PInteger.INSTANCE.getCodec().decodeInt(e.getQualifierArray(), e.getQualifierOffset(), SortOrder.ASC);
+        if (columnQualifier < 0) {
+            throw new IllegalArgumentException("Invalid column qualifier " + columnQualifier + " for cell " + e);
+        }
+        ensureCapacity(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        array[idx] = e;
+        numNonNullElements++;
+        if (firstNonNullElementIdx == -1) {
+            firstNonNullElementIdx = idx;
+        }
+        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;
+                    }
+                }
+                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) {
+        throwUnsupportedOperationException();
+        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<?> c) {
+        throwUnsupportedOperationException();
+        return false;
+    }
+
+    @Override
+    public void clear() {
+        Arrays.fill(array, null);
+    }
+    
+    @Override
+    public Cell get(int index) {
+        rangeCheck(index);
+        int counter = 0;
+        for (; counter < array.length; counter++) {
+            if (array[counter] != null && counter == index) {
+                break;
+            }
+        }
+        return array[counter];
+    }
+
+    @Override
+    public Cell set(int index, Cell element) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public void add(int index, Cell element) {
+        throwUnsupportedOperationException();
+    }
+
+    @Override
+    public Cell remove(int index) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public int indexOf(Object o) {
+        if (o == null) {
+            return -1;
+        } else {
+            for (int i = 0; i < array.length; i++)
+                if (o.equals(array[i])) {
+                    return i;
+                }
+        }
+        return -1;
+    }
+
+    @Override
+    public int lastIndexOf(Object o) {
+        if (o == null) {
+            return -1;
+        }
+        for (int i = array.length - 1; i >=0 ; i--) {
+            if (o.equals(array[i])) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator() {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator(int index) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public List<Cell> subList(int fromIndex, int toIndex) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+    
+    @Override
+    public Iterator<Cell> iterator() {
+        return new Itr();
+    }
+    
+    public Cell getCellForColumnQualifier(int columnQualifier) {
+        int idx = getArrayIndex(columnQualifier);
+        return array[idx];
+    }
+
+    //TODO: samarth need to handle overflow conditions and integer growing beyond sizeofint.
+    private void ensureCapacity(int qualifier) {
+        if (qualifier >= 0 && qualifier < leftBoundary) {
+            // This should happen very rarely.  
+            //TODO: samarth implement this case.
+        } else if (qualifier >= 0 && qualifier > rightBoundary) {
+            // TODO: samarth implement this case.
+        } 
+    }
+
+    private void rangeCheck(int index) {
+        if (index < 0 || index > size() - 1) {
+            throw new IndexOutOfBoundsException();
+        }
+    }
+
+    private void throwUnsupportedOperationException() {
+        throw new UnsupportedOperationException("Operation cannot be supported because it violates invariance");
+    }
+
+    private class Itr implements Iterator<Cell> {
+        private Cell current;
+        private int currentIdx = 0;
+        private boolean exhausted = false;
+        private Itr() {
+            moveToNextNonNullCell(true);
+        }
+
+        @Override
+        public boolean hasNext() {
+            return !exhausted;
+        }
+
+        @Override
+        public Cell next() {
+            if (exhausted) {
+                return null;
+            }
+            Cell next = current;
+            moveToNextNonNullCell(false);
+            return next;
+        }
+
+        @Override
+        public void remove() {
+            throwUnsupportedOperationException();            
+        }
+
+        private void moveToNextNonNullCell(boolean init) {
+            int i = init ? minQualifier : currentIdx + 1;
+            while (i < array.length && (current = array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                currentIdx = i;
+            } else {
+                currentIdx = -1;
+                exhausted = true;
+            }
+        }
+
+    }
+
+    public Cell getFirstCell()  {
+        if (firstNonNullElementIdx == -1) {
+            throw new IllegalStateException("List doesn't have any non-null cell present");
+        }
+        return array[firstNonNullElementIdx];
+    }
+
+    private int getArrayIndex(int columnQualifier) {
+        return columnQualifier - minQualifier;
+    }
+    
+    private class ListItr implements ListIterator<Cell> {
+        private int previousIndex;
+        private int nextIndex;
+        private Cell previous;
+        private Cell next;
+        
+        private ListItr() {
+            movePointersForward(true);
+            previous = null;
+            if (nextIndex != -1) {
+                next = array[nextIndex];
+            }
+        }
+        
+        @Override
+        public boolean hasNext() {
+            return next != null;
+        }
+
+        @Override
+        public Cell next() {
+            Cell toReturn = next;
+            if (toReturn == null) {
+                throw new NoSuchElementException();
+            }
+            movePointersForward(false);
+            return toReturn;
+        }
+
+        @Override
+        public boolean hasPrevious() {
+            return previous != null;
+        }
+
+        @Override
+        public Cell previous() {
+            Cell toReturn = previous;
+            if (toReturn == null) {
+                throw new NoSuchElementException();
+            }
+            movePointersBackward(false);
+            return toReturn;
+        }
+
+        @Override
+        public int nextIndex() {
+            return nextIndex;
+        }
+
+        @Override
+        public int previousIndex() {
+            return previousIndex;
+        }
+
+        @Override
+        public void remove() {
+            // TODO Auto-generated method stub
+            
+        }
+        
+        // TODO: samarth this is one of these ouch methods that can make our implementation frgaile.
+        // It is a non-optional method and can't really be supported 
+        @Override
+        public void set(Cell e) {
+            // TODO Auto-generated method stub
+            
+        }
+
+        @Override
+        public void add(Cell e) {
+            // TODO Auto-generated method stub
+            
+        }
+        
+        private void movePointersForward(boolean init) {
+            int i = init ? 0 : nextIndex;
+            if (!init) {
+                previousIndex = nextIndex;
+                previous = next;
+            } else {
+                previousIndex = -1;
+                previous = null;
+            }
+            while (i < array.length && (array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                nextIndex = i;
+                next = array[i];
+            } else {
+                nextIndex = -1;
+                next = null;
+            }
+        }
+        
+        private void movePointersBackward(boolean init) {
+            int i = init ? 0 : previousIndex;
+        }
+        
+    }
+
+    public static void main (String args[]) throws Exception {
+        UnboundedSkipNullCellsList list = new UnboundedSkipNullCellsList(0, 3); // list of eleven elements
+        System.out.println(list.size());
+        byte[] row = Bytes.toBytes("row");
+        byte[] cf = Bytes.toBytes("cf");
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(0)));
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(5)));
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(10)));
+
+        for (Cell c : list) {
+            System.out.println(c);
+        }
+        System.out.println(list.size());
+        System.out.println(list.get(0));
+        System.out.println(list.get(5));
+        System.out.println(list.get(10));
+        System.out.println(list.get(1));
+        System.out.println(list.remove(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(5))));
+        System.out.println(list.get(5));
+        System.out.println(list.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..48ec277
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/EncodedColumnsUtil.java
@@ -0,0 +1,94 @@
+/*
+ * 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.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 PInteger.INSTANCE.toBytes(column.getEncodedColumnQualifier());
+    }
+
+    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());
+        }
+        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 ? PInteger.INSTANCE.toBytes(column.getEncodedColumnQualifier()) : 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/33ba1ee8/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 7cd7eb3..2635c62 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,14 +18,19 @@
 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 java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
 import java.io.IOException;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
@@ -45,6 +50,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
@@ -54,7 +60,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;
@@ -78,13 +86,16 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
 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;
 
@@ -192,7 +203,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 +230,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 +266,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) {
@@ -286,20 +298,21 @@ public class IndexUtil {
                             if (isEmptyKeyValue(table, ref)) {
                                 return null;
                             }
-                            Map<byte [], List<Cell>> familyMap = dataMutation.getFamilyCellMap();
                             byte[] family = ref.getFamily();
-                            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) {
-                                  ImmutableBytesPtr ptr = new ImmutableBytesPtr();
-                                  kvBuilder.getValueAsPtr(kv, ptr);
-                                  return ptr;
-                                }
+                            RowMutationState rowMutationState = valuesMap.get(ptr);
+                            PColumn column = null;
+                            try {
+                                column = table.getColumnFamily(family).getPColumnForColumnQualifier(qualifier);
+                            } catch (ColumnNotFoundException e) {
+                            } catch (ColumnFamilyNotFoundException e) {
+                            }
+                            if (rowMutationState!=null && column!=null) {
+                                byte[] value = rowMutationState.getColumnValues().get(column);
+                                ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+                                ptr.set(value==null ? ByteUtil.EMPTY_BYTE_ARRAY : value);
+                                SchemaUtil.padData(table.getName().getString(), column, ptr);
+                                return ptr;
                             }
                             return null;
                         }
@@ -312,7 +325,7 @@ public class IndexUtil {
                         regionStartKey = tableRegionLocation.getRegionInfo().getStartKey();
                         regionEndkey = tableRegionLocation.getRegionInfo().getEndKey();
                     }
-                    indexMutations.add(maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, ts, regionStartKey, regionEndkey));
+                    indexMutations.add(maintainer.buildUpdateMutation(kvBuilder, valueGetter, ptr, ts, regionStartKey, regionEndkey, true));
                 }
             }
             return indexMutations;
@@ -426,13 +439,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, PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, SortOrder.getDefault())) 
+                            : new KeyValueColumnExpression(field, family, qualifier);
+                colExpressions[i] = dataColumnExpr;
             }
-            return new TupleProjector(keyValueSchema, keyValueColumns);
+            return new TupleProjector(keyValueSchema, colExpressions);
         }
         return null;
     }
@@ -501,139 +519,142 @@ public class IndexUtil {
             }
             
             // TODO: handle null case (but shouldn't happen)
+            //TODO: samarth confirm if this is the right thing to do here i.e. pass false for look up.
             Tuple joinTuple = new ResultTuple(joinResult);
             // This will create a byte[] that captures all of the values from the data table
             byte[] value =
                     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);
-            // TODO: Create DelegateCell class instead
-            Cell newCell = new Cell() {
+            if (cell != null) {
+                // TODO: Create DelegateCell class instead
+                Cell newCell = new Cell() {
 
-                @Override
-                public byte[] getRowArray() {
-                    return cell.getRowArray();
-                }
+                    @Override
+                    public byte[] getRowArray() {
+                        return cell.getRowArray();
+                    }
 
-                @Override
-                public int getRowOffset() {
-                    return cell.getRowOffset() + offset;
-                }
+                    @Override
+                    public int getRowOffset() {
+                        return cell.getRowOffset() + offset;
+                    }
 
-                @Override
-                public short getRowLength() {
-                    return (short)(cell.getRowLength() - offset);
-                }
+                    @Override
+                    public short getRowLength() {
+                        return (short)(cell.getRowLength() - offset);
+                    }
 
-                @Override
-                public byte[] getFamilyArray() {
-                    return cell.getFamilyArray();
-                }
+                    @Override
+                    public byte[] getFamilyArray() {
+                        return cell.getFamilyArray();
+                    }
 
-                @Override
-                public int getFamilyOffset() {
-                    return cell.getFamilyOffset();
-                }
+                    @Override
+                    public int getFamilyOffset() {
+                        return cell.getFamilyOffset();
+                    }
 
-                @Override
-                public byte getFamilyLength() {
-                    return cell.getFamilyLength();
-                }
+                    @Override
+                    public byte getFamilyLength() {
+                        return cell.getFamilyLength();
+                    }
 
-                @Override
-                public byte[] getQualifierArray() {
-                    return cell.getQualifierArray();
-                }
+                    @Override
+                    public byte[] getQualifierArray() {
+                        return cell.getQualifierArray();
+                    }
 
-                @Override
-                public int getQualifierOffset() {
-                    return cell.getQualifierOffset();
-                }
+                    @Override
+                    public int getQualifierOffset() {
+                        return cell.getQualifierOffset();
+                    }
 
-                @Override
-                public int getQualifierLength() {
-                    return cell.getQualifierLength();
-                }
+                    @Override
+                    public int getQualifierLength() {
+                        return cell.getQualifierLength();
+                    }
 
-                @Override
-                public long getTimestamp() {
-                    return cell.getTimestamp();
-                }
+                    @Override
+                    public long getTimestamp() {
+                        return cell.getTimestamp();
+                    }
 
-                @Override
-                public byte getTypeByte() {
-                    return cell.getTypeByte();
-                }
+                    @Override
+                    public byte getTypeByte() {
+                        return cell.getTypeByte();
+                    }
 
-                @Override
-                public long getMvccVersion() {
-                    return cell.getMvccVersion();
-                }
+                    @Override
+                    public long getMvccVersion() {
+                        return cell.getMvccVersion();
+                    }
 
-                @Override
-                public byte[] getValueArray() {
-                    return cell.getValueArray();
-                }
+                    @Override
+                    public byte[] getValueArray() {
+                        return cell.getValueArray();
+                    }
 
-                @Override
-                public int getValueOffset() {
-                    return cell.getValueOffset();
-                }
+                    @Override
+                    public int getValueOffset() {
+                        return cell.getValueOffset();
+                    }
 
-                @Override
-                public int getValueLength() {
-                    return cell.getValueLength();
-                }
+                    @Override
+                    public int getValueLength() {
+                        return cell.getValueLength();
+                    }
 
-                @Override
-                public byte[] getTagsArray() {
-                    return cell.getTagsArray();
-                }
+                    @Override
+                    public byte[] getTagsArray() {
+                        return cell.getTagsArray();
+                    }
 
-                @Override
-                public int getTagsOffset() {
-                    return cell.getTagsOffset();
-                }
+                    @Override
+                    public int getTagsOffset() {
+                        return cell.getTagsOffset();
+                    }
 
-                @Override
-                public short getTagsLength() {
-                    return cell.getTagsLength();
-                }
+                    @Override
+                    public short getTagsLength() {
+                        return cell.getTagsLength();
+                    }
 
-                @Override
-                public byte[] getValue() {
-                    return cell.getValue();
-                }
+                    @Override
+                    public byte[] getValue() {
+                        return cell.getValue();
+                    }
 
-                @Override
-                public byte[] getFamily() {
-                    return cell.getFamily();
-                }
+                    @Override
+                    public byte[] getFamily() {
+                        return cell.getFamily();
+                    }
 
-                @Override
-                public byte[] getQualifier() {
-                    return cell.getQualifier();
-                }
+                    @Override
+                    public byte[] getQualifier() {
+                        return cell.getQualifier();
+                    }
 
-                @Override
-                public byte[] getRow() {
-                    return cell.getRow();
-                }
+                    @Override
+                    public byte[] getRow() {
+                        return cell.getRow();
+                    }
 
-                @Override
-                @Deprecated
-                public int getTagsLengthUnsigned() {
-                    return cell.getTagsLengthUnsigned();
-                }
-            };
-            // Wrap cell in cell that offsets row key
-            result.set(i, newCell);
+                    @Override
+                    @Deprecated
+                    public int getTagsLengthUnsigned() {
+                        return cell.getTagsLengthUnsigned();
+                    }
+                };
+                // Wrap cell in cell that offsets row key
+                result.set(i, newCell);
+            }
         }
     }
     
@@ -686,4 +707,5 @@ public class IndexUtil {
         }
         return true;
     }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 6027b95..2cc6436 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);
     }
@@ -1028,9 +1028,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/33ba1ee8/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/33ba1ee8/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 b0e8a99..70d7db7 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
@@ -43,6 +43,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 +64,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,5 +898,39 @@ 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 = (Integer)PInteger.INSTANCE.toObject(minQualifier);
+        }
+        byte[] maxQualifier = scan.getAttribute(BaseScannerRegionObserver.MAX_QUALIFIER);
+        if (maxQualifier != null) {
+            maxQ = (Integer)PInteger.INSTANCE.toObject(maxQualifier);
+        }
+        if (minQualifier == null) {
+            return null;
+        }
+        return new Pair<>(minQ, maxQ);
+    }
+    
+    public static boolean useQualifierAsIndex(Pair<Integer, Integer> minMaxQualifiers, boolean isJoin) {
+        return minMaxQualifiers != null;// && isJoin;
+    }
+    
+    public static boolean setMinMaxQualifiersOnScan(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;
+    }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 b53daea..eb55f38 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
@@ -44,8 +44,10 @@ import javax.annotation.Nullable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.KeyValue;
 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.phoenix.exception.DataExceedsCapacityException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
@@ -154,8 +156,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;
     }
     
@@ -394,7 +397,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) {
@@ -591,7 +594,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;
@@ -1020,7 +1023,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;
@@ -1041,4 +1044,21 @@ 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) + ")");
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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 c91b855..ec36ed2 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
@@ -169,7 +169,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();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 2f1a369..7b0a905 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
@@ -67,8 +67,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 +120,9 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-        ColumnExpression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getColumn("ID").getPosition()).newColumnExpression();
+        ColumnExpression 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 +150,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 +941,18 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-
+        PTable table = plan.getTableRef().getTable();
+        ColumnExpression aInteger = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+        ColumnExpression 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 +975,18 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-
+        PTable table = plan.getTableRef().getTable();
+        ColumnExpression aInteger = new ColumnRef(new TableRef(table), table.getPColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+        ColumnExpression 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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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..abcf0c8 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,14 +135,13 @@ 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);
             ImmutableBytesWritable indexKeyPtr = new ImmutableBytesWritable(indexMutation.getRow());
             ptr.set(rowKeyPtr.get(), rowKeyPtr.getOffset(), rowKeyPtr.getLength());
-            byte[] mutablelndexRowKey = im1.buildRowKey(valueGetter, ptr, null, null);
+            byte[] mutablelndexRowKey = im1.buildRowKey(valueGetter, ptr, null, null, false);
             byte[] immutableIndexRowKey = indexKeyPtr.copyBytes();
             assertArrayEquals(immutableIndexRowKey, mutablelndexRowKey);
             for (ColumnReference ref : im1.getCoveredColumns()) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 de5af46..337bcdc 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
@@ -301,11 +301,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) {
@@ -601,7 +601,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 {
@@ -643,17 +643,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);
@@ -685,7 +685,7 @@ public class TestUtil {
                 
                 @Override
                 public SortOrder getSortOrder() {
-                	return SortOrder.getDefault();
+                    return SortOrder.getDefault();
                 }
                 
                 @Override
@@ -711,11 +711,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;
@@ -794,4 +798,3 @@ public class TestUtil {
         conn.createStatement().execute("create table " + tableName + TestUtil.TEST_TABLE_SCHEMA + "TRANSACTIONAL=true");
     }
 }
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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;
 }


[2/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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 b585323..2c7e4bb 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.
@@ -343,7 +389,6 @@ public interface PTable extends PMetaDataEntity {
      */
     int getRowTimestampColPos();
     long getUpdateCacheFrequency();
-
     boolean isNamespaceMapped();
     
     /**
@@ -357,4 +402,77 @@ 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 getValue(String columnFamily) {
+                return null;
+            }
+            
+            @Override
+            public boolean increment(String columnFamily) {
+                return false;
+            }
+        };
+        
+        @Nullable
+        public Integer getValue(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/33ba1ee8/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 7f5efc8..ab0982b 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
@@ -29,6 +29,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;
@@ -39,8 +40,12 @@ 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.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;
@@ -48,13 +53,17 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 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.PInteger;
+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.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
 import org.apache.phoenix.util.StringUtil;
@@ -71,6 +80,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
@@ -98,7 +108,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;
@@ -130,6 +141,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();
@@ -161,8 +174,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();
@@ -176,7 +190,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
@@ -220,7 +234,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 {
@@ -230,7 +244,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, List<PColumn> columns) throws SQLException {
@@ -240,7 +254,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, List<PColumn> columns) throws SQLException {
@@ -250,7 +264,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, List<PColumn> columns, boolean isImmutableRows) throws SQLException {
@@ -260,7 +274,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, List<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled,
@@ -271,7 +285,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 {
@@ -282,7 +296,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 {
@@ -293,7 +307,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 {
@@ -304,7 +318,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,
@@ -313,12 +327,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,
@@ -328,13 +342,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,
@@ -343,11 +357,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
@@ -381,7 +395,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 +
@@ -417,10 +431,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
@@ -446,7 +462,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);
+                                }
+                            }
                         }
                     }
                 }
@@ -510,7 +545,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);
@@ -536,9 +571,9 @@ public class PTableImpl implements PTable {
         for (PName name : this.physicalNames) {
             estimatedSize += name.getEstimatedSize();
         }
-
         this.estimatedSize = estimatedSize;
         this.baseColumnCount = baseColumnCount;
+        this.encodedCQCounter = encodedCQCounter;
     }
 
     @Override
@@ -671,7 +706,7 @@ public class PTableImpl implements PTable {
     }
 
     private PRow newRow(KeyValueBuilder builder, long ts, ImmutableBytesWritable key, int i, byte[]... values) {
-        PRow row = new PRowImpl(builder, key, ts, getBucketNum());
+        PRow row = new PRowImpl(builder, key, ts, getBucketNum(), values.length);
         if (i < values.length) {
             for (PColumnFamily family : getColumnFamilies()) {
                 for (PColumn column : family.getColumns()) {
@@ -696,7 +731,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) {
@@ -715,6 +750,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 = (Integer)PInteger.INSTANCE.toObject(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);
+        }
+    }
 
     /**
      *
@@ -734,8 +801,10 @@ public class PTableImpl implements PTable {
         private Delete unsetValues;
         private Mutation deleteRow;
         private final long ts;
+        // map from column name to value 
+        private Map<PColumn, byte[]> columnToValueMap; 
 
-        public PRowImpl(KeyValueBuilder kvBuilder, ImmutableBytesWritable key, long ts, Integer bucketNum) {
+        public PRowImpl(KeyValueBuilder kvBuilder, ImmutableBytesWritable key, long ts, Integer bucketNum, int numColumns) {
             this.kvBuilder = kvBuilder;
             this.ts = ts;
             if (bucketNum != null) {
@@ -745,7 +814,7 @@ public class PTableImpl implements PTable {
                 this.keyPtr =  new ImmutableBytesPtr(key);
                 this.key = ByteUtil.copyKeyBytesIfNecessary(key);
             }
-
+            this.columnToValueMap = Maps.newHashMapWithExpectedSize(numColumns);
             newMutations();
         }
 
@@ -767,13 +836,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][]; //TODO: samarth probably use the min_qualifier as an offset here.
+                        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);
@@ -808,7 +913,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
             boolean isNull = type.isNull(byteValue);
@@ -820,22 +926,22 @@ public class PTableImpl implements PTable {
             } else if (isNull && !getStoreNulls()) {
                 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 == null ?
                         HConstants.EMPTY_BYTE_ARRAY : 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(name.getString() + "." + column.getName().getString() + " may not exceed " + maxLength + " bytes (" + type.toObject(byteValue) + ")");
-                    }
+                SchemaUtil.padData(name.getString(), column, ptr);
+            	// 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));
             	}
-                removeIfPresent(unsetValues, family, qualifier);
-                addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
-                        column.getFamilyName().getBytesPtr(), column.getName().getBytesPtr(),
-                        ts, ptr));
             }
         }
 
@@ -868,6 +974,11 @@ public class PTableImpl implements PTable {
                 deleteRow.setDurability(Durability.SKIP_WAL);
             }
         }
+        
+        private byte[] getColumnQualifier(PColumn column) {
+            return EncodedColumnsUtil.getColumnQualifier(column, PTableImpl.this);
+        }
+        
     }
 
     @Override
@@ -1028,116 +1139,127 @@ public class PTableImpl implements PTable {
     public IndexType getIndexType() {
         return indexType;
     }
-
+    
+    //FIXME: samarth change the proto here
     /**
      * 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) {
@@ -1215,10 +1337,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() != PTable.EncodedCQCounter.NULL_COUNTER) {
+          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();
     }
 
@@ -1288,4 +1422,14 @@ public class PTableImpl implements PTable {
         } else if (!key.equals(other.key)) 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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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 d5d0b84..6b55756 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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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..4caabbb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/EncodedColumnQualiferCellsList.java
@@ -0,0 +1,553 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+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.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.PTable.StorageScheme;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PInteger;
+
+/**
+ * List implementation that provides indexed based look up when the cell column qualifiers are generated using the
+ * {@link StorageScheme#ENCODED_COLUMN_NAMES} scheme. The api methods in this list assume that the caller wants to see
+ * and add only non null elements in the list. Such an assumption makes the implementation mimic the behavior that one
+ * would get when passing an {@link ArrayList} to hbase for filling in the key values returned by scanners. This
+ * implementation doesn't implement all the optional methods of the {@link List} interface which should be OK. A lot of
+ * things would be screwed up if HBase starts expecting that the the list implementation passed in to scanners
+ * implements all the optional methods of the interface too.
+ * 
+ * For getting elements out o
+ * 
+ * TODO: samarth think about whether we should be using this list with other storage schemes
+ */
+@NotThreadSafe
+public class EncodedColumnQualiferCellsList implements List<Cell> {
+
+    private int minQualifier;
+    private int maxQualifier;
+    private final Cell[] array;
+    private int numNonNullElements;
+    private int firstNonNullElementIdx = -1;
+    private static final int RESERVED_RANGE_MIN = ENCODED_EMPTY_COLUMN_NAME;
+    private static final int RESERVED_RANGE_MAX = ENCODED_CQ_COUNTER_INITIAL_VALUE - 1;
+    private static final String RESERVED_RANGE = "(" + RESERVED_RANGE_MIN + ", " + RESERVED_RANGE_MAX + ")";
+    
+    
+    public EncodedColumnQualiferCellsList(int minQualifier, int maxQualifier) {
+        checkArgument(minQualifier <= maxQualifier, "Invalid arguments. Min: " + minQualifier + ". Max: " + maxQualifier);
+        if (!(minQualifier == maxQualifier && minQualifier == ENCODED_EMPTY_COLUMN_NAME)) {
+            checkArgument(minQualifier >= ENCODED_CQ_COUNTER_INITIAL_VALUE, "Argument minQualifier " + minQualifier + " needs to lie outside of the reserved range: " + RESERVED_RANGE);
+        }
+        this.minQualifier = minQualifier;
+        this.maxQualifier = maxQualifier;
+        int reservedRangeSize = RESERVED_RANGE_MAX - RESERVED_RANGE_MIN + 1;
+        int qualifierRangeSize = minQualifier > RESERVED_RANGE_MAX ? (maxQualifier - minQualifier + 1) : 0;
+        this.array = new Cell[reservedRangeSize + qualifierRangeSize];
+    }
+
+    @Override
+    public int size() {
+        return numNonNullElements;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return numNonNullElements == 0;
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return indexOf(o) >= 0;
+    }
+
+
+    /**
+     * This implementation only returns an array of non-null elements in the list.
+     */
+    @Override
+    public Object[] toArray() {
+        Object[] toReturn = new Object[numNonNullElements];
+        int counter = 0;
+        for (int i = 0; i < array.length; i++) {
+            if (array[i] != null) {
+                toReturn[counter++] = array[i];
+            }
+        }
+        return toReturn;
+    }
+
+
+    /**
+     * This implementation only returns an array of non-null elements in the list.
+     * This is not the most efficient way of copying elemts into an array 
+     */
+    @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 = PInteger.INSTANCE.getCodec().decodeInt(e.getQualifierArray(), e.getQualifierOffset(), SortOrder.ASC);
+        checkQualifierRange(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        array[idx] = e;
+        numNonNullElements++;
+        if (firstNonNullElementIdx == -1) {
+            firstNonNullElementIdx = idx;
+        }
+        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;
+                    }
+                }
+                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) {
+        throwUnsupportedOperationException();
+        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<?> c) {
+        throwUnsupportedOperationException();
+        return false;
+    }
+
+    @Override
+    public void clear() {
+        for (int i = 0; i < array.length; i++) {
+            array[i] = null;
+        }
+        firstNonNullElementIdx = -1;
+        numNonNullElements = 0;
+    }
+    
+    @Override
+    public Cell get(int index) {
+        rangeCheck(index);
+        int numNonNullElementsFound = 0;
+        int i = 0;
+        for (; i < array.length; i++) {
+            if (array[i] != null) {
+                numNonNullElementsFound++;
+                if (numNonNullElementsFound - 1 == index) {
+                    break;
+                }
+            }
+            
+        }
+        return (numNonNullElementsFound - 1) != index ? null : array[i];
+    }
+
+    @Override
+    public Cell set(int index, Cell e) {
+        int columnQualifier = PInteger.INSTANCE.getCodec().decodeInt(e.getQualifierArray(), e.getQualifierOffset(), SortOrder.ASC);
+        checkQualifierRange(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        if (idx != index) {
+            throw new IllegalArgumentException("Attempt made to add cell with encoded column qualifier " + columnQualifier +  "  to the encodedcolumnqualifier list at index " + index);
+        }
+        Cell prev = array[idx];
+        array[idx] = e;
+        numNonNullElements++;
+        if (firstNonNullElementIdx == -1) {
+            firstNonNullElementIdx = idx;
+        }
+        return prev;
+    }
+
+    @Override
+    public void add(int index, Cell element) {
+        throwUnsupportedOperationException();
+    }
+
+    @Override
+    public Cell remove(int index) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public int indexOf(Object o) {
+        if (o == null) {
+            return -1;
+        } else {
+            for (int i = 0; i < array.length; i++)
+                if (o.equals(array[i])) {
+                    return i;
+                }
+        }
+        return -1;
+    }
+
+    @Override
+    public int lastIndexOf(Object o) {
+        if (o == null) {
+            return -1;
+        }
+        for (int i = array.length - 1; i >=0 ; i--) {
+            if (o.equals(array[i])) {
+                return i;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator() {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator(int index) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public List<Cell> subList(int fromIndex, int toIndex) {
+        throwUnsupportedOperationException();
+        return null;
+    }
+    
+    @Override
+    public Iterator<Cell> iterator() {
+        return new Itr();
+    }
+    
+    //TODO: samarth confirm that this method can return null.
+    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 (!(isReservedQualifier(qualifier) || isQualifierInMinMaxRange(qualifier))) { 
+            throw new IndexOutOfBoundsException(
+                "Qualifier " + qualifier + " is out of the valid range. Reserved: " + RESERVED_RANGE + ". Table column qualifier range: ("
+                        + minQualifier + ", " + maxQualifier + ")"); 
+        }
+    }
+    
+    private boolean isReservedQualifier(int qualifier) {
+        return qualifier >= RESERVED_RANGE_MIN && qualifier <= RESERVED_RANGE_MAX; 
+    }
+    
+    private boolean isQualifierInMinMaxRange(int qualifier) {
+        return qualifier >= minQualifier && qualifier <= maxQualifier;
+    }
+
+    private void rangeCheck(int index) {
+        if (index < 0 || index > size() - 1) {
+            throw new IndexOutOfBoundsException();
+        }
+    }
+    
+    private int getArrayIndex(int columnQualifier) {
+        return columnQualifier < ENCODED_CQ_COUNTER_INITIAL_VALUE ? columnQualifier : ENCODED_CQ_COUNTER_INITIAL_VALUE
+                + (columnQualifier - minQualifier);
+    }
+    
+    private void throwUnsupportedOperationException() {
+        throw new UnsupportedOperationException("Operation cannot be supported because it potentially violates the invariance contract of this list implementation");
+    }
+
+    private class Itr implements Iterator<Cell> {
+        private Cell current;
+        private int currentIdx = 0;
+        private boolean exhausted = false;
+        private Itr() {
+            moveToNextNonNullCell(true);
+        }
+
+        @Override
+        public boolean hasNext() {
+            return !exhausted;
+        }
+
+        @Override
+        public Cell next() {
+            if (exhausted) {
+                return null;
+            }
+            Cell next = current;
+            moveToNextNonNullCell(false);
+            return next;
+        }
+
+        @Override
+        public void remove() {
+            throwUnsupportedOperationException();            
+        }
+
+        private void moveToNextNonNullCell(boolean init) {
+            int i = init ? 0 : currentIdx + 1;
+            while (i < array.length && (current = array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                currentIdx = i;
+            } else {
+                currentIdx = -1;
+                exhausted = true;
+            }
+        }
+
+    }
+
+    private class ListItr implements ListIterator<Cell> {
+        private int previousIndex;
+        private int nextIndex;
+        private Cell previous;
+        private Cell next;
+        
+        private ListItr() {
+            movePointersForward(true);
+            previous = null;
+            if (nextIndex != -1) {
+                next = array[nextIndex];
+            }
+        }
+        
+        @Override
+        public boolean hasNext() {
+            return next != null;
+        }
+
+        @Override
+        public Cell next() {
+            Cell toReturn = next;
+            if (toReturn == null) {
+                throw new NoSuchElementException();
+            }
+            movePointersForward(false);
+            return toReturn;
+        }
+
+        @Override
+        public boolean hasPrevious() {
+            return previous != null;
+        }
+
+        @Override
+        public Cell previous() {
+            Cell toReturn = previous;
+            if (toReturn == null) {
+                throw new NoSuchElementException();
+            }
+            movePointersBackward(false);
+            return toReturn;
+        }
+
+        @Override
+        public int nextIndex() {
+            return nextIndex;
+        }
+
+        @Override
+        public int previousIndex() {
+            return previousIndex;
+        }
+
+        @Override
+        public void remove() {
+            // TODO Auto-generated method stub
+            
+        }
+        
+        // TODO: samarth this is one of these ouch methods that can make our implementation frgaile.
+        // It is a non-optional method and can't really be supported 
+        @Override
+        public void set(Cell e) {
+            // TODO Auto-generated method stub
+            
+        }
+
+        @Override
+        public void add(Cell e) {
+            // TODO Auto-generated method stub
+            
+        }
+        
+        private void movePointersForward(boolean init) {
+            int i = init ? 0 : nextIndex;
+            if (!init) {
+                previousIndex = nextIndex;
+                previous = next;
+            } else {
+                previousIndex = -1;
+                previous = null;
+            }
+            while (i < array.length && (array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                nextIndex = i;
+                next = array[i];
+            } else {
+                nextIndex = -1;
+                next = null;
+            }
+        }
+        
+        private void movePointersBackward(boolean init) {
+            int i = init ? 0 : previousIndex;
+        }
+        
+    }
+
+    public static void main (String args[]) throws Exception {
+        EncodedColumnQualiferCellsList list = new EncodedColumnQualiferCellsList(11, 16); // list of 6 elements
+        System.out.println(list.size());
+        
+        byte[] row = Bytes.toBytes("row");
+        byte[] cf = Bytes.toBytes("cf");
+        
+        // add elements in reserved range
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(0)));
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(5)));
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(10)));
+        System.out.println(list.size());
+        for (Cell c : list) {
+            //System.out.println(c);
+        }
+        
+        // add elements in qualifier range
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(12)));
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(14)));
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(16)));
+        System.out.println(list.size());
+        for (Cell c : list) {
+            //System.out.println(c);
+        }
+        
+        list.add(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(11)));
+        System.out.println(list.size());
+        for (Cell c : list) {
+            //System.out.println(c);
+        }
+        
+        System.out.println(list.get(0));
+        System.out.println(list.get(1));
+        System.out.println(list.get(2));
+        System.out.println(list.get(3));
+        System.out.println(list.get(4));
+        System.out.println(list.get(5));
+        System.out.println(list.get(6));
+        System.out.println(list.remove(KeyValue.createFirstOnRow(row, cf, PInteger.INSTANCE.toBytes(5))));
+        System.out.println(list.get(5));
+        System.out.println(list.size());
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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..0c6ae22
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedMultiKeyValueTuple.java
@@ -0,0 +1,88 @@
+/*
+ * 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.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PInteger;
+
+/**
+ * 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 BoundedSkipNullCellsList");
+        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 BoundedSkipNullCellsList");
+        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(PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, SortOrder.ASC));
+    }
+
+    @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;
+    }}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..8f4a846
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedResultTuple.java
@@ -0,0 +1,83 @@
+/*
+ * 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.List;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PInteger;
+
+public class PositionBasedResultTuple extends BaseTuple {
+    private final EncodedColumnQualiferCellsList cells;
+    
+    //TODO: samarth see if we can get rid of this constructor altogether.
+    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 = PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, SortOrder.ASC);
+        return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(cells.getCellForColumnQualifier(columnQualifier));
+    }
+
+    //TODO: samarth implement this.
+    @Override
+    public String toString() {
+      StringBuilder sb = new StringBuilder();
+      sb.append("keyvalues=");
+      return sb.toString();
+    }
+
+    @Override
+    public int size() {
+        return cells.size();
+    }
+
+    @Override
+    public KeyValue getValue(int index) {
+        return org.apache.hadoop.hbase.KeyValueUtil.ensureKeyValue(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;
+    }
+}


[5/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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 262f263..1969bef 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
@@ -584,7 +584,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) {
@@ -618,6 +618,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();
             ImmutableBytesPtr key = rowEntry.getKey();
@@ -625,6 +626,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 {
@@ -656,6 +661,7 @@ public class MutationState implements SQLCloseable {
             if (mutationsPertainingToIndex != null) mutationsPertainingToIndex
                     .addAll(rowMutationsPertainingToIndex);
         }
+        values.putAll(modifiedValues);
     }
     
     /**
@@ -793,7 +799,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());
                     }
                 }
             }
@@ -1461,7 +1467,7 @@ public class MutationState implements SQLCloseable {
             this.rowTsColInfo = rowTsColInfo;
         }
 
-        Map<PColumn, byte[]> getColumnValues() {
+        public Map<PColumn, byte[]> getColumnValues() {
             return columnValues;
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
index f4ff289..087257f 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/SortMergeJoinPlan.java
@@ -656,6 +656,7 @@ public class SortMergeJoinPlan implements QueryPlan {
                 byte[] b = new byte[length];
                 buffer.get(b);
                 Result result = ResultUtil.toResult(new ImmutableBytesWritable(b));
+                //TODO: samarth make joins work with position based look up.
                 return new ResultTuple(result);
             }
             

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 a884949..b6e1de2 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;
@@ -50,9 +53,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/33ba1ee8/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..03b6c61
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayColumnExpression.java
@@ -0,0 +1,110 @@
+/*
+ * 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.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.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+
+/**
+ * 
+ * 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 ColumnExpression {
+    
+    private String displayName; // client-side only
+    private int index;
+    // expression that represents the array (where all cols are stored in a single key value)
+    private KeyValueColumnExpression arrayExpression;
+    // expression that represents this column if (it were stored as a regular key value) 
+    private KeyValueColumnExpression origKVExpression;
+    
+    public ArrayColumnExpression() {
+    }
+    
+    public ArrayColumnExpression(PDatum column, byte[] cf, int index) {
+        super(column);
+        this.index = index;
+        this.arrayExpression = new KeyValueColumnExpression(column, cf, cf);
+    }
+    
+    public ArrayColumnExpression(PColumn column, String displayName, boolean encodedColumnName) {
+        super(column);
+        // array indexes are 1-based TODO: samarth think about the case when the encodedcolumn qualifier is null. Probably best to add a check here for encodedcolumnname to be true
+        this.index = column.getEncodedColumnQualifier() + 1;
+        byte[] cf = column.getFamilyName().getBytes();
+        this.arrayExpression = new KeyValueColumnExpression(column, cf, cf);
+        this.origKVExpression = new KeyValueColumnExpression(column, displayName, encodedColumnName);
+        this.displayName = displayName;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        return PArrayDataType.positionAtArrayElement(tuple, ptr, index, arrayExpression, PVarbinary.INSTANCE, null);
+    }
+
+    @Override
+    public <T> T accept(ExpressionVisitor<T> visitor) {
+        return visitor.visit(this);
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        index = WritableUtils.readVInt(input);
+        arrayExpression = new KeyValueColumnExpression();
+        arrayExpression.readFields(input);
+        origKVExpression = new KeyValueColumnExpression();
+        origKVExpression.readFields(input);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, index);
+        arrayExpression.write(output);
+        origKVExpression.write(output);
+    }
+    
+    public KeyValueColumnExpression getArrayExpression() {
+        return arrayExpression;
+    }
+    
+    public KeyValueColumnExpression getKeyValueExpression() {
+        return origKVExpression;
+    }
+    
+    @Override
+    public String toString() {
+        return displayName;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 d8df29a..f513664 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
@@ -115,6 +115,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;
                         }
                     } else { // No nulls for fixed length
                         oStream.write(ptr.get(), ptr.getOffset(), ptr.getLength());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 75200e8..cdc2261 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
@@ -284,7 +284,8 @@ public enum ExpressionType {
     CeilMonthExpression(CeilMonthExpression.class),
     CeilYearExpression(CeilYearExpression.class),
     DayOfWeekFunction(DayOfWeekFunction.class),
-    DayOfYearFunction(DayOfYearFunction.class);
+    DayOfYearFunction(DayOfYearFunction.class),
+    ArrayColumnExpression(ArrayColumnExpression.class);
     
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..35862c4 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,19 @@ public class KeyValueColumnExpression extends ColumnExpression {
         this.cq = cq;
     }
 
-    public KeyValueColumnExpression(PColumn column, String displayName) {
+    public KeyValueColumnExpression(PColumn column, String displayName, boolean encodedColumnName) {
         super(column);
         this.cf = column.getFamilyName().getBytes();
-        this.cq = column.getName().getBytes();
+        this.cq = EncodedColumnsUtil.getColumnQualifier(column, encodedColumnName);
         this.displayName = displayName;
     }
 
     public byte[] getColumnFamily() {
         return cf;
     }
-
-    public byte[] getColumnName() {
+    
+    //TODO: samarth look for the callers of this.
+    public byte[] getColumnQualifier() {
         return cq;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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);
     }


[4/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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/33ba1ee8/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
new file mode 100644
index 0000000..7ca6d9e
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/ReplaceArrayColumnWithKeyValueColumnExpressionVisitor.java
@@ -0,0 +1,37 @@
+/*
+ * 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.visitor;
+
+import java.util.List;
+
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.expression.Expression;
+
+public class ReplaceArrayColumnWithKeyValueColumnExpressionVisitor extends CloneExpressionVisitor {
+
+    @Override
+    public boolean isCloneNode(Expression node, List<Expression> children) {
+        return !children.equals(node.getChildren());
+    }
+
+    @Override
+    public Expression visit(ArrayColumnExpression node) {
+        return node.getKeyValueExpression();
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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 b8b0350..d1f6211 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
@@ -40,7 +40,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
@@ -53,6 +53,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() {
 
@@ -60,10 +62,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
@@ -87,6 +91,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));
@@ -110,12 +117,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 {
@@ -153,9 +161,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/33ba1ee8/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..5909286 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,12 @@ 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;
+            }
+            @Override
+            public Void visit(ArrayColumnExpression expression) {
+                inputTuple.addColumn(expression.getArrayExpression().getColumnFamily(), expression.getArrayExpression().getColumnQualifier());
                 return null;
             }
         };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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..527b948 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,13 @@ public abstract class SingleKeyValueComparisonFilter extends BooleanExpressionFi
             @Override
             public Void visit(KeyValueColumnExpression expression) {
                 cf = expression.getColumnFamily();
-                cq = expression.getColumnName();
+                cq = expression.getColumnQualifier();
+                return null;
+            }
+            @Override
+            public Void visit(ArrayColumnExpression expression) {
+                cf = expression.getArrayExpression().getColumnFamily();
+                cq = expression.getArrayExpression().getColumnQualifier();
                 return null;
             }
         };

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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 6595562..dc5e3e7 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
@@ -32,6 +32,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,6 +45,7 @@ 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;
@@ -51,10 +53,14 @@ import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.IndexExpressionCompiler;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.expression.CoerceExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.expression.ArrayConstructorExpression;
 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.expression.visitor.ReplaceArrayColumnWithKeyValueColumnExpressionVisitor;
 import org.apache.phoenix.hbase.index.ValueGetter;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -67,14 +73,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,17 +90,22 @@ 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.PInteger;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.BitSet;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.ExpressionUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 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 +289,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 +310,46 @@ 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;
-
+        /* 
+         * There is nothing to prevent new indexes on existing tables to have encoded column names.
+         * Except, 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. 
+         */
+        this.usesEncodedColumnNames = EncodedColumnsUtil.usesEncodedColumnNames(dataTable);
         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 +360,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 +393,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 +402,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 +424,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 +439,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 +455,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 +464,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,19 +513,20 @@ 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)  {
+    
+    public byte[] buildRowKey(ValueGetter valueGetter, ImmutableBytesWritable rowKeyPtr, byte[] regionStartKey, byte[] regionEndKey, boolean convertArrayColToKeyValueCol)  {
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         boolean prependRegionStartKey = isLocalIndex && regionStartKey != null;
         boolean isIndexSalted = !isLocalIndex && nIndexSaltBuckets > 0;
@@ -523,6 +595,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                 SortOrder dataSortOrder;
                 if (dataPkPosition[i] == EXPRESSION_NOT_PRESENT) {
                 	Expression expression = expressionIterator.next();
+                	if (convertArrayColToKeyValueCol) {
+                	    expression = expression.accept(new ReplaceArrayColumnWithKeyValueColumnExpressionVisitor());
+                	}
                 	dataColumnType = expression.getDataType();
                 	dataSortOrder = expression.getSortOrder();
                     isNullable = expression.isNullable();
@@ -855,36 +930,84 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         return indexRowKeySchema;
     }
 
-    public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] regionStartKey, byte[] regionEndKey) throws IOException {
+    public Put buildUpdateMutation(KeyValueBuilder kvBuilder, ValueGetter valueGetter, ImmutableBytesWritable dataRowKeyPtr, long ts, byte[] regionStartKey, byte[] regionEndKey, boolean convertArrayColToKeyValueCol) throws IOException {
         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);
+            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey, convertArrayColToKeyValueCol);
             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) {
+        byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey, convertArrayColToKeyValueCol);
+        ImmutableBytesPtr rowKey = new ImmutableBytesPtr(indexRowKey);
+        if (storeColsInSingleCell) {
+            // map from column family to list of columns (for covered columns)
+            Map<String, List<ColumnReference>> familyToColListMap = Maps.newHashMap();
+            for (ColumnReference ref : this.getCoveredColumns()) {
+                String cf = Bytes.toString(ref.getFamily());
+                if (!familyToColListMap.containsKey(cf)) {
+                    familyToColListMap.put(cf, Lists.<ColumnReference>newArrayList());
+                }
+                familyToColListMap.get(cf).add(ref);
+            }
+            // iterate over each column family and create a byte[] containing all the columns 
+            for (Entry<String, List<ColumnReference>> entry : familyToColListMap.entrySet()) {
+                byte[] columnFamily = entry.getKey().getBytes();
+                List<ColumnReference> colRefs = entry.getValue();
+                int maxIndex = Integer.MIN_VALUE;
+                // find the max col qualifier
+                for (ColumnReference colRef : colRefs) {
+                    byte[] qualifier = this.coveredColumnsMap.get(colRef).getQualifier();
+                    maxIndex = Math.max(maxIndex, PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, SortOrder.getDefault()));
+                }
+                byte[][] colValues = new byte[maxIndex+1][];
+                // set the values of the columns
+                for (ColumnReference colRef : colRefs) {
+                    ImmutableBytesWritable value = valueGetter.getLatestValue(colRef);
+                    if (value != null) {
+                        byte[] qualifier = this.coveredColumnsMap.get(colRef).getQualifier();
+                        int index = PInteger.INSTANCE.getCodec().decodeInt(qualifier, 0, SortOrder.getDefault());
+                        colValues[index] = value.get();
+                    }
+                }
+                
+                List<Expression> children = Lists.newArrayListWithExpectedSize(colRefs.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));
                 }
             }
         }
@@ -964,7 +1087,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     
     @SuppressWarnings("deprecation")
     public Delete buildDeleteMutation(KeyValueBuilder kvBuilder, ValueGetter oldState, ImmutableBytesWritable dataRowKeyPtr, Collection<KeyValue> pendingUpdates, long ts, byte[] regionStartKey, byte[] regionEndKey) throws IOException {
-        byte[] indexRowKey = this.buildRowKey(oldState, dataRowKeyPtr, regionStartKey, regionEndKey);
+        byte[] indexRowKey = this.buildRowKey(oldState, dataRowKeyPtr, regionStartKey, regionEndKey, false);
         // Delete the entire row if any of the indexed columns changed
         DeleteType deleteType = null;
         if (oldState == null || (deleteType=getDeleteTypeOrNull(pendingUpdates)) != null || hasIndexedColumnChanged(oldState, pendingUpdates)) { // Deleting the entire row
@@ -973,14 +1096,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 +1122,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 +1185,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 +1220,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 +1275,21 @@ 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)));
+        }
         initCachedState();
     }
     
@@ -1171,9 +1313,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 +1330,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 +1343,16 @@ 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());
+        }
     }
 
     public int getEstimatedByteSize() {
@@ -1241,16 +1399,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,11 +1409,19 @@ 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;
                 }
+                @Override
+                public Void visit(ArrayColumnExpression expression) {
+					KeyValueColumnExpression colExpression = expression.getArrayExpression();
+					if (indexedColumns.add(new ColumnReference(colExpression.getColumnFamily(), colExpression.getColumnQualifier()))) {
+                		indexedColumnTypes.add(colExpression.getDataType());
+                	}
+					return null;
+                }
             };
             expression.accept(visitor);
         }
@@ -1523,4 +1682,16 @@ 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;
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
index 9d2955b..b1454b7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexCodec.java
@@ -74,7 +74,7 @@ public class PhoenixIndexCodec extends BaseIndexCodec {
             indexUpdate.setTable(maintainer.isLocalIndex() ? state.getEnvironment().getRegion()
                     .getTableDesc().getName() : maintainer.getIndexTableName());
             Put put = maintainer.buildUpdateMutation(KV_BUILDER, valueGetter, ptr, state.getCurrentTimestamp(), env
-                    .getRegion().getRegionInfo().getStartKey(), env.getRegion().getRegionInfo().getEndKey());
+                    .getRegion().getRegionInfo().getStartKey(), env.getRegion().getRegionInfo().getEndKey(), false);
             indexUpdate.setUpdate(put);
             indexUpdates.add(indexUpdate);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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 2685b93..0d6d881 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,16 @@
  */
 package org.apache.phoenix.iterate;
 
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.EXPECTED_UPPER_REGION_KEY;
 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.util.ByteUtil.EMPTY_BYTE_ARRAY;
+import static org.apache.phoenix.util.ScanUtil.setMinMaxQualifiersOnScan;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
@@ -49,6 +53,9 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 
 import org.apache.hadoop.hbase.HConstants;
+
+import javax.management.Query;
+
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HRegionLocation;
 import org.apache.hadoop.hbase.TableName;
@@ -82,14 +89,17 @@ 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.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.stats.GuidePostsInfo;
 import org.apache.phoenix.schema.stats.PTableStats;
 import org.apache.phoenix.schema.stats.StatisticsUtil;
+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.LogUtil;
 import org.apache.phoenix.util.PrefixByteCodec;
 import org.apache.phoenix.util.PrefixByteDecoder;
@@ -208,7 +218,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 +236,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 +247,77 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                         new DistinctPrefixFilter(plan.getTableRef().getTable().getRowKeySchema(),
                                 cols));
             }
-
+            //TODO: samarth add condition to not do position based look ups in case of joins so that we won't need to do the hacky check inside co-processors.
+            if (setMinMaxQualifiersOnScan(table)) {
+                Pair<Integer, Integer> minMaxQualifiers = getMinMaxQualifiers(scan, context);
+                if (minMaxQualifiers != null) {
+                    scan.setAttribute(BaseScannerRegionObserver.MIN_QUALIFIER, PInteger.INSTANCE.toBytes(minMaxQualifiers.getFirst()));
+                    scan.setAttribute(BaseScannerRegionObserver.MAX_QUALIFIER, PInteger.INSTANCE.toBytes(minMaxQualifiers.getSecond()));
+                }
+            }
             if (optimizeProjection) {
                 optimizeProjection(context, scan, table, statement);
             }
         }
     }
-
+    
+    private static Pair<Integer, Integer> getMinMaxQualifiers(Scan scan, StatementContext context) {
+        PTable table = context.getCurrentTable().getTable();
+        StorageScheme storageScheme = table.getStorageScheme();
+		checkArgument(EncodedColumnsUtil.usesEncodedColumnNames(storageScheme), "Method should only be used for tables using encoded column names");
+        Integer minQualifier = null;
+        Integer maxQualifier = null;
+        boolean emptyKVProjected = false;
+        for (Pair<byte[], byte[]> whereCol : context.getWhereConditionColumns()) {
+            byte[] cq = whereCol.getSecond();
+            if (cq != null) {
+                int qualifier = (Integer)PInteger.INSTANCE.toObject(cq);
+                if (qualifier == ENCODED_EMPTY_COLUMN_NAME) {
+                    emptyKVProjected = true;
+                    continue;
+                }
+                if (minQualifier == null && maxQualifier == null) {
+                    minQualifier = maxQualifier = qualifier;
+                } else {
+                    if (qualifier < minQualifier) {
+                        minQualifier = qualifier;
+                    } else if (qualifier > maxQualifier) {
+                        maxQualifier = qualifier;
+                    }
+                }
+            }
+        }
+        Map<byte[], NavigableSet<byte[]>> familyMap = scan.getFamilyMap();
+        for (Entry<byte[], NavigableSet<byte[]>> entry : familyMap.entrySet()) {
+            if (entry.getValue() != null) {
+                for (byte[] cq : entry.getValue()) {
+                    if (cq != null) {
+                        int qualifier = (Integer)PInteger.INSTANCE.toObject(cq);
+                        if (qualifier == ENCODED_EMPTY_COLUMN_NAME) {
+                            emptyKVProjected = true;
+                            continue;
+                        }
+                        if (minQualifier == null && maxQualifier == null) {
+                            minQualifier = maxQualifier = qualifier;
+                        } else {
+                            if (qualifier < minQualifier) {
+                                minQualifier = qualifier;
+                            } else if (qualifier > maxQualifier) {
+                                maxQualifier = qualifier;
+                            }
+                        }
+                    }
+                }
+            }
+        }
+        if (minQualifier == null && emptyKVProjected) {
+            return new Pair<>(ENCODED_EMPTY_COLUMN_NAME, ENCODED_EMPTY_COLUMN_NAME);
+        } else if (minQualifier == null) {
+            return null;
+        }
+        return new Pair<>(minQualifier, maxQualifier);
+    }
+    
     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 +414,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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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..531bbe7 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, boolean isJoin) {
         this.scanner = scanner;
+        this.useQualifierAsIndex = ScanUtil.useQualifierAsIndex(minMaxQualifiers, isJoin);
+        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/33ba1ee8/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 5a4a791..e4b32b1 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;
 
@@ -315,6 +315,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;
@@ -588,9 +595,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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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);
         }


[6/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..e884439 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,17 +89,18 @@ 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();
+        // TODO: samarth this is likely just an in memory reference for compilation purposes. Probably ok to pass non-encoded scheme and null counter.
         PTable tempTable = PTableImpl.makePTable(statement.getConnection().getTenantId(),
             UNION_SCHEMA_NAME, UNION_TABLE_NAME, PTableType.SUBQUERY, null,
             HConstants.LATEST_TIMESTAMP, scn == null ? HConstants.LATEST_TIMESTAMP : scn,
             null, null, projectedColumns, null, null, null, true, null, null, null, true,
             true, true, null, null, null, false, false, 0, 0L,
             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/33ba1ee8/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..63ad9c0 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
@@ -29,6 +29,8 @@ import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.ArrayColumnExpression;
+import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Determinism;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
@@ -51,11 +53,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 +172,14 @@ public class WhereCompiler {
         public Expression visit(ColumnParseNode node) throws SQLException {
             ColumnRef ref = resolveColumn(node);
             TableRef tableRef = ref.getTableRef();
+            ColumnExpression 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 +200,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 +228,22 @@ public class WhereCompiler {
 
             }
         }
+        
+        public void increment(ArrayColumnExpression column) {
+            switch (count) {
+                case NONE:
+                    count = Count.SINGLE;
+                    this.column = column.getArrayExpression();
+                    break;
+                case SINGLE:
+                    count = column.getArrayExpression().equals(this.column) ? Count.SINGLE : Count.MULTIPLE;
+                    break;
+                case MULTIPLE:
+                    break;
+
+            }
+        }
+        
         public Count getCount() {
             return count;
         }
@@ -256,6 +278,12 @@ public class WhereCompiler {
                     counter.increment(expression);
                     return null;
                 }
+                
+                @Override
+                public Void visit(ArrayColumnExpression expression) {
+                    counter.increment(expression);
+                    return null;
+                }
             });
             switch (counter.getCount()) {
             case NONE:

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 5320971..64e5efd 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";
@@ -100,6 +104,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
@@ -306,14 +312,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);
     }
     
     /**
@@ -331,7 +337,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,
@@ -339,7 +345,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();
@@ -436,11 +442,14 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                             tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
                     }
                     if (projector != null) {
-                        Tuple tuple = projector.projectResults(new ResultTuple(Result.create(result)));
+                        // TODO: samarth think if this is the right thing to do here.
+                        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;
@@ -474,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)
@@ -527,24 +537,10 @@ 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
+                // TODO: samarth how does this dummy column qualifier play with encoded column names
                 result.add(new KeyValue(rowKv.getRowArray(), rowKv.getRowOffset(), rowKv.getRowLength(),
                         QueryConstants.ARRAY_VALUE_COLUMN_FAMILY, 0, QueryConstants.ARRAY_VALUE_COLUMN_FAMILY.length,
                         QueryConstants.ARRAY_VALUE_COLUMN_QUALIFIER, 0,

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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..d21508a 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), j != null);
         if (ScanUtil.isLocalIndex(scan) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
@@ -139,7 +145,7 @@ 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) {
@@ -155,9 +161,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
         }
         if (keyOrdered) { // Optimize by taking advantage that the rows are
                           // already in the required group by key order
-            return scanOrdered(c, scan, innerScanner, expressions, aggregators, limit);
+            return scanOrdered(c, scan, innerScanner, expressions, aggregators, limit, j != null);
         } else { // Otherwse, collect them all up in an in memory map
-            return scanUnordered(c, scan, innerScanner, expressions, aggregators, limit);
+            return scanUnordered(c, scan, innerScanner, expressions, aggregators, limit, j != null);
         }
     }
 
@@ -363,7 +369,7 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
      */
     private RegionScanner scanUnordered(ObserverContext<RegionCoprocessorEnvironment> c, Scan scan,
             final RegionScanner scanner, final List<Expression> expressions,
-            final ServerAggregators aggregators, long limit) throws IOException {
+            final ServerAggregators aggregators, long limit, boolean isJoin) throws IOException {
         if (logger.isDebugEnabled()) {
             logger.debug(LogUtil.addCustomAnnotations("Grouped aggregation over unordered rows with scan " + scan
                     + ", group by " + expressions + ", aggregators " + aggregators, ScanUtil.getCustomAnnotations(scan)));
@@ -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), isJoin);
         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.
@@ -445,12 +450,14 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
      */
     private RegionScanner scanOrdered(final ObserverContext<RegionCoprocessorEnvironment> c,
             final Scan scan, final RegionScanner scanner, final List<Expression> expressions,
-            final ServerAggregators aggregators, final long limit) throws IOException {
+            final ServerAggregators aggregators, final long limit, final boolean isJoin) throws IOException {
 
         if (logger.isDebugEnabled()) {
             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, isJoin);
         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
@@ -511,6 +518,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                             KeyValueUtil.newKeyValue(currentKey.get(), currentKey.getOffset(),
                                 currentKey.getLength(), SINGLE_COLUMN_FAMILY, SINGLE_COLUMN,
                                 AGG_TIMESTAMP, value, 0, value.length);
+                    //TODO: samarth aaha how do we handle this. It looks like we are adding stuff like this to the results
+                    // that we are returning. Bounded skip null cell list won't handle this properly. Interesting. So how do we
+                    // handle this. Does having a reserved set of column qualifiers help here? 
                     results.add(keyValue);
                     if (logger.isDebugEnabled()) {
                         logger.debug(LogUtil.addCustomAnnotations("Adding new aggregate row: "

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..1b55c0d 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
@@ -109,7 +109,7 @@ public class HashJoinRegionScanner implements RegionScanner {
     private void processResults(List<Cell> result, boolean hasBatchLimit) throws IOException {
         if (result.isEmpty())
             return;
-        
+        //TODO: samarth make joins work with position based lookup.
         Tuple tuple = new ResultTuple(Result.create(result));
         // For backward compatibility. In new versions, HashJoinInfo.forceProjection()
         // always returns true.

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 5ab42b9..a7247e2 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;
@@ -190,8 +193,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 +214,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 +289,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 +316,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 +347,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 +361,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 +374,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 +392,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 +731,10 @@ 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 :
+            PInteger.INSTANCE.getCodec().decodeInt(columnQualifierKV.getValueArray(), columnQualifierKV.getValueOffset(), SortOrder.getDefault());
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false, columnQualifier);
         columns.add(column);
     }
     
@@ -926,37 +942,50 @@ 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;
+            }
+            //TODO: samarth remember why we have colums present at LINK_TYPE_INDEX
+            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 +993,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);
@@ -1991,7 +2028,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return result;
                 }
                 region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet());
-                // Invalidate from cache
+                // Invalidate from cache. //TODO: samarth should we invalidate the base table from the cache too here.
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
                 }
@@ -2164,6 +2201,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)));
@@ -2198,8 +2236,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) {
@@ -2326,26 +2364,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;
     }
@@ -2503,8 +2541,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.
@@ -2570,7 +2608,9 @@ 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)) {
+                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()) {
@@ -2813,7 +2853,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;
@@ -3066,7 +3106,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;
@@ -3155,10 +3195,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.
@@ -3179,9 +3220,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/33ba1ee8/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..5cfb102 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, boolean isJoin) {
         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), isJoin);
             return new OrderedResultIterator(inner, orderByExpressions, thresholdBytes, limit >= 0 ? limit : null, null,
                     estimatedRowSize);
         } catch (IOException e) {
@@ -218,10 +218,12 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        //TODO: samarth get rid of this join shit. Joins should support position based look up.
+        boolean useQualifierAsIndex = ScanUtil.useQualifierAsIndex(ScanUtil.getMinMaxQualifiersFromScan(scan), j != null) && 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) {
@@ -229,10 +231,10 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         }
         if (scanOffset != null) {
             innerScanner = getOffsetScanner(c, innerScanner,
-                    new OffsetResultIterator(new RegionScannerResultIterator(innerScanner), scanOffset),
+                    new OffsetResultIterator(new RegionScannerResultIterator(innerScanner, ScanUtil.getMinMaxQualifiersFromScan(scan), j != null), scanOffset),
                     scan.getAttribute(QueryConstants.LAST_SCAN) != null);
         }
-        final OrderedResultIterator iterator = deserializeFromScan(scan,innerScanner);
+        final OrderedResultIterator iterator = deserializeFromScan(scan, innerScanner, j != null);
         if (iterator == null) {
             return innerScanner;
         }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 a312020..3129ef8 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;
@@ -49,10 +50,8 @@ import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.RegionTooBusyException;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
-import org.apache.hadoop.hbase.client.HTableInterface;
 import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.client.Scan;
@@ -64,7 +63,6 @@ import org.apache.hadoop.hbase.regionserver.InternalScanner;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
 import org.apache.hadoop.hbase.regionserver.ScanType;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.WritableUtils;
@@ -98,7 +96,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;
@@ -114,6 +115,7 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TimeKeeper;
+import org.apache.tephra.TxConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -121,8 +123,6 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
-import org.apache.tephra.TxConstants;
-
 
 /**
  * Region observer that aggregates ungrouped rows(i.e. SQL query with aggregation function and no GROUP BY).
@@ -300,6 +300,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;
@@ -315,12 +316,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);//TODO: samarth check this
         }
         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), j != null) && scan.getAttribute(BaseScannerRegionObserver.TOPN) != null;
         if ((localIndexScan && !isDelete && !isDescRowKeyOrderUpgrade) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
@@ -329,7 +332,7 @@ 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)  {
@@ -369,7 +372,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)));
         }
@@ -386,7 +390,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
@@ -494,7 +498,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                         Put put = maintainer.buildUpdateMutation(kvBuilder,
                                             valueGetter, ptr, results.get(0).getTimestamp(),
                                             env.getRegion().getRegionInfo().getStartKey(),
-                                            env.getRegion().getRegionInfo().getEndKey());
+                                            env.getRegion().getRegionInfo().getEndKey(), false);
                                         indexMutations.add(put);
                                     }
                                 }
@@ -589,8 +593,10 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                     if (!timeStamps.contains(kvts)) {
                                         Put put = new Put(kv.getRowArray(), kv.getRowOffset(),
                                             kv.getRowLength());
-                                        put.add(emptyCF, QueryConstants.EMPTY_COLUMN_BYTES, kvts,
-                                            ByteUtil.EMPTY_BYTE_ARRAY);
+                                        // FIXME: Use the right byte array value. Transactional tables can't
+                                        // have empty byte arrays since Tephra seems them as delete markers.
+                                        put.add(emptyCF, emptyKVQualifier != null ? emptyKVQualifier
+                                                : QueryConstants.EMPTY_COLUMN_BYTES, kvts, ByteUtil.EMPTY_BYTE_ARRAY);
                                         mutations.add(put);
                                     }
                                 }


[7/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Posted by sa...@apache.org.
Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98


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

Branch: refs/heads/encodecolumns2
Commit: 33ba1ee8e6bdadae5973c4f364b2c3c4cca53303
Parents: 08d9c71
Author: Samarth <sa...@salesforce.com>
Authored: Wed Oct 5 00:11:07 2016 -0700
Committer: Samarth <sa...@salesforce.com>
Committed: Wed Oct 5 00:11:07 2016 -0700

----------------------------------------------------------------------
 .../apache/phoenix/end2end/AlterTableIT.java    |  251 +++-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |    4 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |   22 +
 .../phoenix/end2end/PhoenixRuntimeIT.java       |    4 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |    2 +-
 .../phoenix/end2end/StatsCollectorIT.java       |    4 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   45 +-
 .../phoenix/end2end/index/DropMetadataIT.java   |   13 +-
 .../end2end/index/IndexExpressionIT.java        |   21 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |   26 +-
 .../phoenix/end2end/index/IndexTestUtil.java    |   13 +-
 .../phoenix/compile/CreateTableCompiler.java    |    2 +-
 .../phoenix/compile/ExpressionCompiler.java     |   18 +-
 .../apache/phoenix/compile/FromCompiler.java    |   54 +-
 .../apache/phoenix/compile/JoinCompiler.java    |    8 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |    3 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |   11 +-
 .../phoenix/compile/ProjectionCompiler.java     |   10 +-
 .../apache/phoenix/compile/QueryCompiler.java   |    2 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    2 +-
 .../compile/TupleProjectionCompiler.java        |   22 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    6 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   36 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   42 +-
 .../coprocessor/DelegateRegionScanner.java      |    5 +
 .../GroupedAggregateRegionObserver.java         |   36 +-
 .../coprocessor/HashJoinRegionScanner.java      |    2 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  171 ++-
 .../phoenix/coprocessor/ScanRegionObserver.java |   12 +-
 .../coprocessor/SequenceRegionObserver.java     |    1 +
 .../UngroupedAggregateRegionObserver.java       |   28 +-
 .../coprocessor/generated/PTableProtos.java     | 1379 ++++++++++++++++--
 .../apache/phoenix/execute/BaseQueryPlan.java   |   25 +-
 .../apache/phoenix/execute/MutationState.java   |   12 +-
 .../phoenix/execute/SortMergeJoinPlan.java      |    1 +
 .../apache/phoenix/execute/TupleProjector.java  |    6 +-
 .../expression/ArrayColumnExpression.java       |  110 ++
 .../expression/ArrayConstructorExpression.java  |    1 +
 .../phoenix/expression/ExpressionType.java      |    3 +-
 .../expression/KeyValueColumnExpression.java    |   16 +-
 .../phoenix/expression/LiteralExpression.java   |   11 +-
 .../expression/ProjectedColumnExpression.java   |    1 +
 .../visitor/CloneExpressionVisitor.java         |    6 +
 .../expression/visitor/ExpressionVisitor.java   |    2 +
 ...lumnWithKeyValueColumnExpressionVisitor.java |   37 +
 .../StatelessTraverseAllExpressionVisitor.java  |    7 +-
 .../StatelessTraverseNoExpressionVisitor.java   |    7 +-
 .../phoenix/filter/ColumnProjectionFilter.java  |   24 +-
 .../filter/MultiKeyValueComparisonFilter.java   |   10 +-
 .../SingleCQKeyValueComparisonFilter.java       |    3 +-
 .../filter/SingleKeyValueComparisonFilter.java  |   10 +-
 .../apache/phoenix/hbase/index/ValueGetter.java |    1 +
 .../example/CoveredColumnIndexCodec.java        |    1 -
 .../apache/phoenix/index/IndexMaintainer.java   |  305 +++-
 .../apache/phoenix/index/PhoenixIndexCodec.java |    2 +-
 .../index/PhoenixTransactionalIndexer.java      |   16 +-
 .../phoenix/iterate/BaseResultIterators.java    |   83 +-
 .../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      |    2 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    1 -
 .../apache/phoenix/query/QueryConstants.java    |   57 +-
 .../phoenix/query/QueryServicesOptions.java     |    1 -
 .../org/apache/phoenix/schema/ColumnRef.java    |   10 +-
 .../apache/phoenix/schema/DelegateColumn.java   |    5 +
 .../apache/phoenix/schema/DelegateTable.java    |   20 +-
 .../apache/phoenix/schema/KeyValueSchema.java   |    2 +
 .../apache/phoenix/schema/MetaDataClient.java   |  339 ++++-
 .../java/org/apache/phoenix/schema/PColumn.java |    9 +-
 .../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  |  124 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  434 ++++--
 .../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   |  553 +++++++
 .../schema/tuple/MultiKeyValueTuple.java        |    1 +
 .../tuple/PositionBasedMultiKeyValueTuple.java  |   88 ++
 .../schema/tuple/PositionBasedResultTuple.java  |   83 ++
 .../phoenix/schema/tuple/ResultTuple.java       |   35 +-
 .../org/apache/phoenix/schema/tuple/Tuple.java  |    4 +
 .../tuple/UnboundedSkipNullCellsList.java       |  488 +++++++
 .../apache/phoenix/util/EncodedColumnsUtil.java |   94 ++
 .../java/org/apache/phoenix/util/IndexUtil.java |  268 ++--
 .../org/apache/phoenix/util/PhoenixRuntime.java |    8 +-
 .../org/apache/phoenix/util/ResultUtil.java     |   60 -
 .../java/org/apache/phoenix/util/ScanUtil.java  |   38 +
 .../org/apache/phoenix/util/SchemaUtil.java     |   28 +-
 .../phoenix/compile/HavingCompilerTest.java     |    2 +-
 .../phoenix/compile/QueryCompilerTest.java      |    2 +-
 .../phoenix/compile/WhereCompilerTest.java      |   26 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   10 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   10 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |    5 +-
 .../expression/ColumnExpressionTest.java        |   16 +-
 .../phoenix/index/IndexMaintainerTest.java      |    5 +-
 .../iterate/AggregateResultScannerTest.java     |    2 +-
 .../query/BaseConnectionlessQueryTest.java      |   18 +-
 .../phoenix/query/ConnectionlessTest.java       |    2 -
 .../java/org/apache/phoenix/util/TestUtil.java  |   43 +-
 phoenix-protocol/src/main/PTable.proto          |    8 +
 115 files changed, 5137 insertions(+), 981 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..0acbe1e 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,12 +62,14 @@ 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;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 /**
@@ -231,8 +242,19 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals(exists, rs.next());
     }
 
+    
+    @Test
+    @Ignore //FIXME: See PHOENIX-3340 
+    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 +266,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 +2140,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 +2149,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 +2228,217 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 		}
 	}
 	
+	@Test
+	public void testClientAndServerMetadataForEncodedColumns() 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.getValue(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().getValue(DEFAULT_COLUMN_FAMILY));
+	        assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), baseTable.getEncodedCQCounter().getValue("A"));
+	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getValue(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 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 + 2), encodedCqCounter.getValue(DEFAULT_COLUMN_FAMILY));
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1), encodedCqCounter.getValue("B"));
+            
+            // assert that the server side metadata is updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2, true);
+            assertEncodedCQCounter("B", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, 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);
+            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 + 3, true);
+            assertEncodedCQCounter("A", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, true);
+            assertEncodedCQCounter("B", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1, 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 + 2);
+            assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+            // 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 + 4), encodedCqCounter.getValue(DEFAULT_COLUMN_FAMILY));
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2), encodedCqCounter.getValue("A"));
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2), encodedCqCounter.getValue("B"));
+            
+            // 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().getValue(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 + 4, true);
+            assertEncodedCQCounter("A", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2, true);
+            assertEncodedCQCounter("B", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2, true);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL3", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+            assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+            assertEncodedCQValue("A", "VIEW_COL4", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+            assertEncodedCQValue("B", "VIEW_COL5", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+            // 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 + 5), encodedCqCounter.getValue(DEFAULT_COLUMN_FAMILY));
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3), encodedCqCounter.getValue("A"));
+            assertEquals((Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2), encodedCqCounter.getValue("B"));
+            
+            // 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().getValue(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 + 5), true);
+            assertEncodedCQCounter("A", schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 3), true);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL10", schemaName, viewName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 4));
+            assertEncodedCQValue("A", "COL11", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+            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/33ba1ee8/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..e4baa20 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" : "");
@@ -679,4 +679,4 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
         } 
     }
     
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..e367866 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
@@ -18,7 +18,17 @@
 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.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
@@ -26,6 +36,8 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.List;
@@ -35,10 +47,15 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.EncodedCQCounter;
+import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.SchemaNotFoundException;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -502,4 +519,9 @@ public class CreateTableIT extends BaseClientManagedTimeIT {
             fail();
         }
     }
+    
+    @Test
+    public void testCreateTableIfNotExistsForEncodedColumnNames() throws Exception {
+        
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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 2445948..687c26a 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
@@ -124,13 +124,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);
         

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 21ccbf4..02ad55b 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/33ba1ee8/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..5d36a2f 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/33ba1ee8/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..7cca633 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
@@ -870,8 +870,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 +1246,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 +1264,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 +1361,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/33ba1ee8/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/33ba1ee8/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
index ba04ad7..a2888cf 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
@@ -39,15 +39,15 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
-import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.PRow;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.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/33ba1ee8/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 3928f66..70b04d5 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
@@ -300,7 +300,7 @@ public class CreateTableCompiler {
         @Override
         public Boolean visit(KeyValueColumnExpression node) {
             try {
-                this.position = table.getColumnFamily(node.getColumnFamily()).getColumn(node.getColumnName()).getPosition();
+                this.position = table.getColumnFamily(node.getColumnFamily()).getPColumnForColumnQualifier(node.getColumnQualifier()).getPosition();
             } catch (SQLException e) {
                 throw new RuntimeException(e); // Impossible
             }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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..0e6ffb6 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,8 @@ 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);
+                //TODO: samarth confirm if passing false here is the right thing to do. I think it is since it is a mapped view, but still.
+                return new SingleTableColumnResolver(connection, tableNode, e.getTimeStamp(), new HashMap<String, UDFParseNode>(1), false, false);
             }
             throw e;
         }
@@ -257,7 +261,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 +336,27 @@ 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(),
+            //TODO: samarth should we change the ptableimpl constructor here to set non-encoded column name scheme and null counter
+            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 +452,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 +676,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);
@@ -774,16 +780,18 @@ public class FromCompiler {
                     // referenced by an outer wild-card select.
                     alias = String.valueOf(position);
                 }
+                //TODO: samarth confirm this is the right change i.e. using null for column qualifier
                 PColumnImpl column = new PColumnImpl(PNameFactory.newName(alias),
                         PNameFactory.newName(QueryConstants.DEFAULT_COLUMN_FAMILY),
-                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false);
+                        null, 0, 0, true, position++, SortOrder.ASC, null, null, false, null, false, false, null);
                 columns.add(column);
             }
+            //TODO: samarth confirm if the storage scheme and encode cqcounters should be passed as null here. Could it be non-encoded column names?
             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 +866,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 +879,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/33ba1ee8/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/33ba1ee8/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..d7aa13f 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 =
@@ -165,6 +165,7 @@ public class ListJarsQueryPlan implements QueryPlan {
                                 Type.Put.getCode(), HConstants.EMPTY_BYTE_ARRAY);
                     List<Cell> cells = new ArrayList<Cell>(1);
                     cells.add(cell);
+                    //TODO: samarth confirm if passing false is the right thing to do here.
                     return new ResultTuple(Result.create(cells));
                 } catch (IOException e) {
                     throw new SQLException(e);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 e43b596..45225a8 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, 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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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/33ba1ee8/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..9f7de36 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,10 @@ 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());
     }
-
+    
+    //TODO: samarth does it matter what the storage scheme here is for the projected tables?
     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 +177,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


[3/7] phoenix git commit: Branch rebased to 08d9c7154194de69ce688ad78357bffb8d34c92f of 4.x-HBase-0.98

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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 d41636a..731049e 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
@@ -92,6 +92,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;
@@ -568,7 +569,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 6ddfe9a..6c01597 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/33ba1ee8/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 ace228b..73d96a3 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;
@@ -118,7 +121,7 @@ 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;
 
 
 /**
@@ -149,23 +152,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 = PInteger.INSTANCE.toBytes(1);
+    
+    public static final byte[] ARRAY_VALUE_COLUMN_FAMILY = RESERVED_COLUMN_FAMILY_BYTES;
+    public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = PInteger.INSTANCE.toBytes(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 = PInteger.INSTANCE.toBytes(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 +202,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 = PInteger.INSTANCE.toBytes(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 +239,13 @@ 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;
+    
+    //TODO: samarth think about this more.
+    /**
+     * 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.
@@ -288,6 +312,9 @@ public interface QueryConstants {
             IS_NAMESPACE_MAPPED + " BOOLEAN," +
             AUTO_PARTITION_SEQ + " VARCHAR," +
             APPEND_ONLY_SCHEMA + " BOOLEAN," +
+            ENCODED_COLUMN_QUALIFIER + " INTEGER," +
+            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 +390,7 @@ public interface QueryConstants {
     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();
-
+    //TODO: samarth we may or may not have to tweak this constant for short/long/int column qualifer schemes.
+    //Qualifier used for the cell that stores various column values using the storage scheme StorageScheme#COLUMNS_STORED_IN_SINGLE_CELL
+    //public static final ImmutableBytesPtr SINGLE_CELL_COLUMN_QUALIFIER = new ImmutableBytesPtr(PInteger.INSTANCE.toBytes(ENCODED_CQ_COUNTER_INITIAL_VALUE)); 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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/33ba1ee8/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 76f6218..ed0c711 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
@@ -18,10 +18,13 @@
 package org.apache.phoenix.schema;
 
 import org.apache.http.annotation.Immutable;
+import org.apache.phoenix.expression.ArrayColumnExpression;
 import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.schema.PTable.StorageScheme;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
 
@@ -45,7 +48,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) {
@@ -109,7 +112,10 @@ public class ColumnRef {
         	return new ProjectedColumnExpression(column, table, displayName);
         }
        
-        return new KeyValueColumnExpression(column, displayName);
+        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/33ba1ee8/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 a60229e..4ac8f46 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
@@ -90,4 +90,9 @@ public class DelegateColumn extends DelegateDatum implements PColumn {
 	public boolean isDynamic() {
 		return getDelegate().isDynamic();
 	}
+
+    @Override
+    public Integer getEncodedColumnQualifier() {
+        return getDelegate().getEncodedColumnQualifier();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 3ee012f..9874866 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
@@ -280,4 +280,20 @@ public class DelegateTable implements PTable {
     public boolean isAppendOnlySchema() {
         return delegate.isAppendOnlySchema();
     }
+    
+    @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/33ba1ee8/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
index 1ab8c86..1bcf808 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/KeyValueSchema.java
@@ -109,6 +109,8 @@ public class KeyValueSchema extends ValueSchema {
             Field field = fields.get(i);
             PDataType type = field.getDataType();
             for (int j = 0; j < field.getCount(); j++) {
+                //TODO: samarth it is at this point that we are looking up stuff in the result tuple to figure out
+                // where exactly the value is here.
                 if (expressions[index].evaluate(tuple, ptr) && ptr.getLength() > 0) { // Skip null values
                     if (index >= minNullableIndex) {
                         valueSet.set(index - minNullableIndex);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 1e635d8..1f2f1cc 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.IMMUTABLE_ROWS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
@@ -67,6 +69,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;
@@ -83,9 +86,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;
@@ -185,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.PTableStats;
 import org.apache.phoenix.schema.types.PDataType;
@@ -198,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;
@@ -256,8 +264,9 @@ public class MetaDataClient {
                     UPDATE_CACHE_FREQUENCY + "," +
                     IS_NAMESPACE_MAPPED + "," +
                     AUTO_PARTITION_SEQ +  "," +
-                    APPEND_ONLY_SCHEMA +
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    APPEND_ONLY_SCHEMA + "," +
+                    STORAGE_SCHEME +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -272,6 +281,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 + "," +
@@ -281,6 +291,16 @@ public class MetaDataClient {
                     LINK_TYPE + "," +
                     PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
                     ") VALUES (?, ?, ?, ?, ?, ?)";
+    
+    private static final String UPDATE_ENCODED_COLUMN_COUNTER = 
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+            TENANT_ID + ", " + 
+            TABLE_SCHEM + "," +
+            TABLE_NAME + "," +
+            COLUMN_FAMILY + "," +
+            COLUMN_QUALIFIER_COUNTER + 
+            ") VALUES (?, ?, ?, ?, ?)";
+
     private static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
@@ -334,8 +354,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 + "," +
@@ -355,8 +376,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 + "," +
@@ -701,22 +723,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()) {
@@ -726,17 +748,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;
@@ -803,8 +825,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();
     }
@@ -823,7 +850,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();
@@ -871,15 +898,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());
@@ -997,14 +1023,14 @@ public class MetaDataClient {
                 final List<PName> physicalNames = Lists.newArrayListWithExpectedSize(2);
                 if (table.isMultiTenant() || MetaDataUtil.hasViewIndexTable(connection, table.getPhysicalName())) {
                     names.add(PNameFactory.newName(SchemaUtil.getTableName(
-                            MetaDataUtil.getViewIndexSchemaName(table.getSchemaName().getString()),
-                            MetaDataUtil.getViewIndexTableName(table.getTableName().getString()))));
+                        MetaDataUtil.getViewIndexSchemaName(table.getSchemaName().getString()),
+                        MetaDataUtil.getViewIndexTableName(table.getTableName().getString()))));
                     physicalNames.add(PNameFactory.newName(MetaDataUtil.getViewIndexPhysicalName(table.getPhysicalName().getBytes())));
                 }
                 if (MetaDataUtil.hasLocalIndexTable(connection, table.getPhysicalName())) {
                     names.add(PNameFactory.newName(SchemaUtil.getTableName(
-                            MetaDataUtil.getLocalIndexSchemaName(table.getSchemaName().getString()),
-                            MetaDataUtil.getLocalIndexTableName(table.getTableName().getString()))));
+                        MetaDataUtil.getLocalIndexSchemaName(table.getSchemaName().getString()),
+                        MetaDataUtil.getLocalIndexTableName(table.getTableName().getString()))));
                     physicalNames.add(PNameFactory.newName(MetaDataUtil.getLocalIndexPhysicalName(table.getPhysicalName().getBytes())));
                 }
                 int i = 0;
@@ -1764,7 +1790,7 @@ public class MetaDataClient {
                 updateCacheFrequency = updateCacheFrequencyProp;
             }
             String autoPartitionSeq = (String) TableProperty.AUTO_PARTITION_SEQ.getValue(tableProps);
-
+            
             Boolean storeNullsProp = (Boolean) TableProperty.STORE_NULLS.getValue(tableProps);
             if (storeNullsProp == null) {
                 if (parent == null) {
@@ -1786,7 +1812,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;
                 }
@@ -1801,7 +1827,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();
@@ -1932,7 +1958,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()) {
@@ -1976,7 +2002,79 @@ 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.
+                 */
+                //TODO: samarth these checks for whether table exists need to be changed for local indexes. Hate having all of these special cases for local indexes.
+                // It is making the code unmaintainable. For local indexes, the physical table already exists.
+                byte[] tableNameBytes = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
+                boolean tableExists = true;
+                try (HBaseAdmin admin = connection.getQueryServices().getAdmin()) {
+                    try {
+                        admin.getTableDescriptor(tableNameBytes);
+                    } catch (org.apache.hadoop.hbase.TableNotFoundException e) {
+                        tableExists = false;
+                    }
+                } catch (IOException e) {
+                    throw new RuntimeException(e);
+                } catch (UnsupportedOperationException e) {
+                    //FIXME: samarth I am not sure about this.
+                }
+                if (parent != null) {
+                    storageScheme = parent.getStorageScheme();
+                } else if (tableExists) {
+                    storageScheme = StorageScheme.NON_ENCODED_COLUMN_NAMES;
+                } else 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...
@@ -1995,7 +2093,15 @@ 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 familyName = isPkColumn ? null : (colDefFamily != null ? colDefFamily : (defaultFamilyName != null ? defaultFamilyName : DEFAULT_COLUMN_FAMILY));
+                Integer encodedCQ =  isPkColumn ? null : cqCounter.getValue(familyName);   
+                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false, encodedCQ);
+                if (cqCounter.increment(familyName)) {
+                    changedCqCounters.put(familyName, cqCounter.getValue(familyName));
+                }
                 if (SchemaUtil.isPKColumn(column)) {
                     // TODO: remove this constraint?
                     if (pkColumnsIterator.hasNext() && !column.getName().getString().equals(pkColumnsIterator.next().getFirst().getColumnName())) {
@@ -2030,6 +2136,38 @@ public class MetaDataClient {
                         column.getFamilyName());
                 }
             }
+            
+            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();
+                }
+            }
+            
             // 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)
@@ -2111,13 +2249,14 @@ public class MetaDataClient {
             if (SchemaUtil.isMetaTable(schemaName,tableName)) {
                 // TODO: what about stats for system catalog?
                 PName newSchemaName = PNameFactory.newName(schemaName);
+                // Column names and qualifiers and hardcoded for system tables.
                 PTable table = PTableImpl.makePTable(tenantId,newSchemaName, PNameFactory.newName(tableName), tableType,
                         null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                         PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns.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);
             }
 
@@ -2145,14 +2284,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);
@@ -2230,6 +2369,7 @@ public class MetaDataClient {
                 tableUpsert.setString(23, autoPartitionSeq);
             }
             tableUpsert.setBoolean(24, isAppendOnlySchema);
+            tableUpsert.setByte(25, storageScheme.getSerializedValue()); //TODO: samarth should there be a null check here?
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2323,12 +2463,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;
@@ -2338,6 +2484,14 @@ public class MetaDataClient {
         }
     }
 
+    private static boolean isPkColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef, ColumnName columnDefName) {
+        return colDef.isPK() || (pkConstraint != null && pkConstraint.getColumnWithSortOrder(columnDefName) != null);
+    }
+    
+    private static boolean incrementEncodedCQCounter(StorageScheme storageScheme, ColumnDef colDef) {
+        return storageScheme != StorageScheme.NON_ENCODED_COLUMN_NAMES && !colDef.isPK();
+    }
+
     /**
      * A table can be a parent table to tenant-specific tables if all of the following conditions are true:
      * <p>
@@ -2726,7 +2880,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;
@@ -2745,7 +2898,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);
                             }
@@ -2756,7 +2909,7 @@ public class MetaDataClient {
                     }
                     else {
                         try {
-                            table.getColumn(columnName);
+                            table.getPColumnForColumnName(columnName);
                             if (!ifNotExists) {
                                 throw new ColumnAlreadyExistsException(schemaName, tableName, columnName);
                             }
@@ -2807,7 +2960,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()) {
@@ -2897,10 +3051,14 @@ 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 ) {
+                    //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) {
@@ -2920,11 +3078,23 @@ public class MetaDataClient {
                                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.ROWTIMESTAMP_CREATE_ONLY)
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                             }
-                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
+                            Integer encodedCQ = null;
+                            if (!colDef.isPK()) {
+                                String colDefFamily = colDef.getColumnDefName().getFamilyName();
+                                //FIXME: samarth Think about local indexes. They have a different column family
+                                String familyName = (colDefFamily != null ? colDefFamily : (tableForCQCounters
+                                        .getDefaultFamilyName() != null ? tableForCQCounters.getDefaultFamilyName()
+                                        .getString() : DEFAULT_COLUMN_FAMILY));
+                                encodedCQ = cqCounterToUse.getValue(familyName);   
+                                if (cqCounterToUse.increment(familyName)) {
+                                    changedCqCounters.put(familyName, cqCounterToUse.getValue(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;
@@ -2936,13 +3106,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));
                                 }
@@ -2951,14 +3121,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);
                                     }
                                 }
@@ -2994,9 +3164,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);
+                if (changingPhoenixTableProperty || numCols > 0) { 
+                    seqNum = incrementTableSeqNum(table, tableType, numCols, isTransactional, updateCacheFrequency, isImmutableRows, disableWAL, multiTenant, storeNulls);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -3005,6 +3176,38 @@ public class MetaDataClient {
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
                 tableMetaData.addAll(columnMetaData);
+                boolean sharedIndex = tableType == PTableType.INDEX && (table.getIndexType() == IndexType.LOCAL || table.getViewIndexId() != null);
+                String tenantIdToUse = connection.getTenantId() != null && sharedIndex ? connection.getTenantId().getString() : null;
+                //TODO: samarth I am not sure this is going to work on server side. But for now lets add these mutations here.
+                if (!changedCqCounters.isEmpty()) {
+                    PreparedStatement linkStatement;
+                    //TODO: samarth i don't think we need the shared index check here.
+                    //if (!sharedIndex) {
+                        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;
 
@@ -3231,23 +3434,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()));
@@ -3337,13 +3540,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/33ba1ee8/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
index 0f5fa44..a2fadac 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
@@ -27,7 +27,7 @@ package org.apache.phoenix.schema;
 public interface PColumn extends PDatum {
 
     /**
-     * @return the name of the column qualifier
+     * @return the name of the column
      */
     PName getName();
 
@@ -60,4 +60,11 @@ public interface PColumn extends PDatum {
     boolean isRowTimestamp();
     
     boolean isDynamic();
+    
+    /**
+     * @return name of the HBase column qualifier
+     * TODO: samarth I think we should should change this to return byte[] array.
+     * Then we won't have to worry about calling SchemaUtil... everywhere 
+     */
+    Integer getEncodedColumnQualifier();
 }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/33ba1ee8/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 8d7161e..5dd3321 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(), null, oldColumn.isRowTimestamp(), oldColumn.isDynamic());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), null, oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getEncodedColumnQualifier());
                 columns.add(newColumn);
             }