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 2017/02/27 06:22:16 UTC

[01/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Repository: phoenix
Updated Branches:
  refs/heads/4.x-HBase-1.1 03c2bc2cd -> d6a82e29f


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 089c5f1..4571115 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
@@ -95,8 +95,8 @@ public class ConnectionlessTest {
         "    entity_history_id char(12) not null,\n" + 
         "    created_by varchar,\n" + 
         "    created_date date\n" +
-        "    CONSTRAINT pk PRIMARY KEY (organization_id, key_prefix, entity_history_id) ) " +
-        (saltBuckets == null ? "" : (PhoenixDatabaseMetaData.SALT_BUCKETS + "=" + saltBuckets));
+        "    CONSTRAINT pk PRIMARY KEY (organization_id, key_prefix, entity_history_id) ) COLUMN_ENCODED_BYTES=4 " +
+        (saltBuckets == null ? "" : " , " + (PhoenixDatabaseMetaData.SALT_BUCKETS + "=" + saltBuckets));
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(getUrl(), props);
         PreparedStatement statement = conn.prepareStatement(dmlStmt);
@@ -141,31 +141,29 @@ public class ConnectionlessTest {
         assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey1, kv.getRow());        
-        assertEquals(name1, PVarchar.INSTANCE.toObject(kv.getValue()));
-        assertTrue(iterator.hasNext());
+        assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue()));
         kv = iterator.next();
         assertArrayEquals(expectedRowKey1, kv.getRow());        
-        assertEquals(now, PDate.INSTANCE.toObject(kv.getValue()));
+        assertEquals(name1, PVarchar.INSTANCE.toObject(kv.getValue()));
         assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey1, kv.getRow());        
-        assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue()));
+        assertEquals(now, PDate.INSTANCE.toObject(kv.getValue()));
     }
 
     private static void assertRow2(Iterator<KeyValue> iterator, byte[] expectedRowKey2) {
         KeyValue kv;
-        assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey2, kv.getRow());        
-        assertEquals(name2, PVarchar.INSTANCE.toObject(kv.getValue()));
+        assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue()));
         assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey2, kv.getRow());        
-        assertEquals(now, PDate.INSTANCE.toObject(kv.getValue()));
+        assertEquals(name2, PVarchar.INSTANCE.toObject(kv.getValue()));
         assertTrue(iterator.hasNext());
         kv = iterator.next();
         assertArrayEquals(expectedRowKey2, kv.getRow());        
-        assertEquals(QueryConstants.EMPTY_COLUMN_VALUE, PVarchar.INSTANCE.toObject(kv.getValue()));
+        assertEquals(now, PDate.INSTANCE.toObject(kv.getValue()));
     }
     
     @Test

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
new file mode 100644
index 0000000..d8c5cdb
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/ImmutableStorageSchemeTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.expression.DelegateExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.ByteUtil;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+
+@RunWith(Parameterized.class)
+public class ImmutableStorageSchemeTest {
+    
+    protected static final LiteralExpression CONSTANT_EXPRESSION = LiteralExpression.newConstant(QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+    protected static final byte[] BYTE_ARRAY1 = new byte[]{1,2,3,4,5};
+    protected static final byte[] BYTE_ARRAY2 = new byte[]{6,7,8};
+    protected Expression FALSE_EVAL_EXPRESSION = new DelegateExpression(LiteralExpression.newConstant(null)) {
+        @Override
+        public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+            return false;
+        }
+    };
+    private ImmutableStorageScheme immutableStorageScheme;
+    
+    @Parameters(name="ImmutableStorageSchemeTest_immutableStorageScheme={0}}") // name is used by failsafe as file name in reports
+    public static ImmutableStorageScheme[] data() {
+        ImmutableStorageScheme[] values = ImmutableStorageScheme.values();
+        // skip ONE_CELL_PER_COLUMN
+        return Arrays.copyOfRange(values, 1, values.length);
+    }
+    
+    public ImmutableStorageSchemeTest(ImmutableStorageScheme immutableStorageScheme) {
+        this.immutableStorageScheme = immutableStorageScheme;
+    }
+
+    @Test
+    public void testWithExpressionsThatEvaluatetoFalse() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        children.add(CONSTANT_EXPRESSION);
+        children.add(FALSE_EVAL_EXPRESSION);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
+        children.add(FALSE_EVAL_EXPRESSION);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
+        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        singleCellConstructorExpression.evaluate(null, ptr);
+        
+        ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+        ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
+        assertTrue(decoder.decode(ptrCopy, 0));
+        assertArrayEquals(QueryConstants.EMPTY_COLUMN_VALUE_BYTES, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertFalse(decoder.decode(ptrCopy, 1));
+        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, 2));
+        assertArrayEquals(BYTE_ARRAY1, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertFalse(decoder.decode(ptrCopy, 3));
+        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, 4));
+        assertArrayEquals(BYTE_ARRAY2, ptrCopy.copyBytesIfNecessary());
+    }
+    
+    @Test
+    public void testWithMaxOffsetLargerThanShortMax() throws Exception {
+        int numElements = Short.MAX_VALUE+2;
+        List<Expression> children = Lists.newArrayListWithExpectedSize(numElements);
+        for (int i=0; i<numElements; ++i) {
+            children.add(CONSTANT_EXPRESSION);
+        }
+        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        singleCellConstructorExpression.evaluate(null, ptr);
+
+        ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+        ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
+        assertTrue(decoder.decode(ptrCopy, 0));
+        assertArrayEquals(QueryConstants.EMPTY_COLUMN_VALUE_BYTES, ptrCopy.copyBytesIfNecessary());
+        
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, 14999));
+        assertArrayEquals(QueryConstants.EMPTY_COLUMN_VALUE_BYTES, ptrCopy.copyBytesIfNecessary());
+        
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, numElements-1));
+        assertArrayEquals(QueryConstants.EMPTY_COLUMN_VALUE_BYTES, ptrCopy.copyBytesIfNecessary());
+    }
+    
+    @Test
+    public void testWithMaxOffsetSmallerThanShortMin() throws Exception {
+        int numElements = Short.MAX_VALUE+2;
+        List<Expression> children = Lists.newArrayListWithExpectedSize(numElements);
+        for (int i=0; i<=numElements; i+=2) {
+            children.add(CONSTANT_EXPRESSION);
+            children.add(FALSE_EVAL_EXPRESSION);
+        }
+        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        singleCellConstructorExpression.evaluate(null, ptr);
+
+        ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+        ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
+        assertTrue(decoder.decode(ptrCopy, 0));
+        assertArrayEquals(QueryConstants.EMPTY_COLUMN_VALUE_BYTES, ptrCopy.copyBytesIfNecessary());
+        
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertFalse(decoder.decode(ptrCopy, 1));
+        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+        
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, numElements-1));
+        assertArrayEquals(QueryConstants.EMPTY_COLUMN_VALUE_BYTES, ptrCopy.copyBytesIfNecessary());
+        
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertFalse(decoder.decode(ptrCopy, numElements));
+        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+    }
+    
+    @Test
+    public void testLeadingNulls() throws Exception {
+        List<Expression> children = Lists.newArrayListWithExpectedSize(4);
+        LiteralExpression nullExpression = LiteralExpression.newConstant(null);
+        children.add(nullExpression);
+        children.add(nullExpression);
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY1, PVarbinary.INSTANCE));
+        children.add(LiteralExpression.newConstant(BYTE_ARRAY2, PVarbinary.INSTANCE));
+        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        singleCellConstructorExpression.evaluate(null, ptr);
+        
+        ImmutableBytesPtr ptrCopy = new ImmutableBytesPtr(ptr);
+        ColumnValueDecoder decoder = immutableStorageScheme.getDecoder();
+        assertTrue(decoder.decode(ptrCopy, 0));
+        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, 1));
+        assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, 2));
+        assertArrayEquals(BYTE_ARRAY1, ptrCopy.copyBytesIfNecessary());
+        ptrCopy = new ImmutableBytesPtr(ptr);
+        assertTrue(decoder.decode(ptrCopy, 3));
+        assertArrayEquals(BYTE_ARRAY2, ptrCopy.copyBytesIfNecessary());
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeForArraysTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeForArraysTest.java b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeForArraysTest.java
index 333fbf9..2aeeeb8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeForArraysTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/schema/types/PDataTypeForArraysTest.java
@@ -324,7 +324,7 @@ public class PDataTypeForArraysTest {
 				PVarchar.INSTANCE, strArr);
 		byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
 		int offset = ptr.getOffset();
 		int length = ptr.getLength();
 		byte[] bs = ptr.get();
@@ -342,7 +342,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 0, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 0, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -365,7 +365,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -402,7 +402,7 @@ public class PDataTypeForArraysTest {
 				PVarchar.INSTANCE, strArr);
 		byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-		PArrayDataType.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+		PArrayDataTypeDecoder.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
 		int offset = ptr.getOffset();
 		int length = ptr.getLength();
 		byte[] bs = ptr.get();
@@ -423,7 +423,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 2, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 2, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -444,7 +444,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 2, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 2, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -464,7 +464,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -485,7 +485,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -506,7 +506,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -528,7 +528,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -548,7 +548,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -569,7 +569,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 3, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -590,7 +590,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 0, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 0, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -611,7 +611,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -632,7 +632,7 @@ public class PDataTypeForArraysTest {
                 PVarchar.INSTANCE, strArr);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 4, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
         int offset = ptr.getOffset();
         int length = ptr.getLength();
         byte[] bs = ptr.get();
@@ -649,7 +649,7 @@ public class PDataTypeForArraysTest {
 				PVarchar.INSTANCE, strArr);
 		byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-		PArrayDataType.positionAtArrayElement(ptr, 0, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+		PArrayDataTypeDecoder.positionAtArrayElement(ptr, 0, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
 		int offset = ptr.getOffset();
 		int length = ptr.getLength();
 		byte[] bs = ptr.get();
@@ -667,7 +667,7 @@ public class PDataTypeForArraysTest {
 				PVarchar.INSTANCE, strArr);
 		byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-		PArrayDataType.positionAtArrayElement(ptr, 1, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
+		PArrayDataTypeDecoder.positionAtArrayElement(ptr, 1, PVarchar.INSTANCE, PVarchar.INSTANCE.getByteSize());
 		int offset = ptr.getOffset();
 		int length = ptr.getLength();
 		byte[] bs = ptr.get();
@@ -688,7 +688,7 @@ public class PDataTypeForArraysTest {
 		PLongArray.INSTANCE.toObject(arr, PLongArray.INSTANCE);
 		byte[] bytes = PLongArray.INSTANCE.toBytes(arr);
 		ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-		PArrayDataType.positionAtArrayElement(ptr, 2, PLong.INSTANCE, PLong.INSTANCE.getByteSize());
+		PArrayDataTypeDecoder.positionAtArrayElement(ptr, 2, PLong.INSTANCE, PLong.INSTANCE.getByteSize());
 		int offset = ptr.getOffset();
 		int length = ptr.getLength();
 		byte[] bs = ptr.get();
@@ -1196,7 +1196,7 @@ public class PDataTypeForArraysTest {
         PhoenixArray arr = new PhoenixArray(PVarchar.INSTANCE, objects);
         byte[] bytes = PVarcharArray.INSTANCE.toBytes(arr, PVarchar.INSTANCE, SortOrder.DESC);
         ImmutableBytesWritable ptr = new ImmutableBytesWritable(bytes);
-        PArrayDataType.positionAtArrayElement(ptr, 2, PVarchar.INSTANCE, null);
+        PArrayDataTypeDecoder.positionAtArrayElement(ptr, 2, PVarchar.INSTANCE, null);
         String value = (String)PVarchar.INSTANCE.toObject(ptr, SortOrder.DESC);
         assertEquals(null, value);
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
index 430c20b..7b81c8d 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/PhoenixRuntimeTest.java
@@ -26,6 +26,8 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.sql.Connection;
 import java.sql.Date;
 import java.sql.DriverManager;
@@ -37,6 +39,7 @@ import java.util.List;
 import java.util.Properties;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.expression.Expression;
@@ -45,6 +48,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.types.PDataType;
 import org.junit.Test;
@@ -342,5 +346,6 @@ public class PhoenixRuntimeTest extends BaseConnectionlessQueryTest {
         long skewedTs = ts + QueryConstants.MILLIS_IN_DAY; // skew of a day
         // Even with a day of skew, we won't consider the ts a nanos timestamp
         assertEquals(skewedTs, PhoenixRuntime.getWallClockTimeFromCellTimeStamp(skewedTs));
-    }    
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/util/QualifierEncodingSchemeTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/QualifierEncodingSchemeTest.java b/phoenix-core/src/test/java/org/apache/phoenix/util/QualifierEncodingSchemeTest.java
new file mode 100644
index 0000000..2b08d7d
--- /dev/null
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/QualifierEncodingSchemeTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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 org.apache.phoenix.schema.PTable.QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.ONE_BYTE_QUALIFIERS;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.THREE_BYTE_QUALIFIERS;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.TWO_BYTE_QUALIFIERS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme.InvalidQualifierBytesException;
+import org.junit.Test;
+
+public class QualifierEncodingSchemeTest {
+    
+    @Test
+    public void testOneByteQualifierEncodeDecode() {
+        assertEquals(1, ONE_BYTE_QUALIFIERS.decode(ONE_BYTE_QUALIFIERS.encode(1)));
+        assertEquals(127, ONE_BYTE_QUALIFIERS.decode(ONE_BYTE_QUALIFIERS.encode(127)));
+        assertEquals(63, ONE_BYTE_QUALIFIERS.decode(ONE_BYTE_QUALIFIERS.encode(63)));
+        assertEquals(130, ONE_BYTE_QUALIFIERS.decode(ONE_BYTE_QUALIFIERS.encode(130)));
+        assertEquals(255, ONE_BYTE_QUALIFIERS.decode(ONE_BYTE_QUALIFIERS.encode(255)));
+        byte[] arr1 = ONE_BYTE_QUALIFIERS.encode(255);
+        byte[] arr2 = new byte[] {-128, arr1[0]};
+        assertEquals(255, ONE_BYTE_QUALIFIERS.decode(arr2, 1, 1));
+        try {
+            ONE_BYTE_QUALIFIERS.decode(arr2);
+            fail();
+        } catch (InvalidQualifierBytesException expected) {}
+        try {
+            ONE_BYTE_QUALIFIERS.decode(arr2, 0, 2);
+            fail();
+        } catch (InvalidQualifierBytesException expected) {}
+        
+    }
+    
+    @Test
+    public void testTwoByteQualifierEncodeDecode() {
+        assertEquals(1, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(1)));
+        assertEquals(127, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(127)));
+        assertEquals(63, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(63)));
+        assertEquals(130, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(130)));
+        assertEquals(128, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(128)));
+        assertEquals(129, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(129)));
+        assertEquals(32767, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(32767)));
+        assertEquals(32768, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(32768)));
+        assertEquals(65535, TWO_BYTE_QUALIFIERS.decode(TWO_BYTE_QUALIFIERS.encode(65535)));
+        byte[] arr1 = TWO_BYTE_QUALIFIERS.encode(65535);
+        byte[] arr2 = new byte[] {-128, arr1[0], arr1[1]};
+        assertEquals(65535, TWO_BYTE_QUALIFIERS.decode(arr2, 1, 2));
+        try {
+            TWO_BYTE_QUALIFIERS.decode(arr2);
+            fail();
+        } catch (InvalidQualifierBytesException expected) {}
+    }
+    
+    @Test
+    public void testThreeByteQualifierEncodeDecode() {
+        assertEquals(1, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(1)));
+        assertEquals(127, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(127)));
+        assertEquals(63, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(63)));
+        assertEquals(130, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(130)));
+        assertEquals(128, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(128)));
+        assertEquals(129, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(129)));
+        assertEquals(32767, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(32767)));
+        assertEquals(32768, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(32768)));
+        assertEquals(65535, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(65535)));
+        assertEquals(16777215, THREE_BYTE_QUALIFIERS.decode(THREE_BYTE_QUALIFIERS.encode(16777215)));
+        byte[] arr1 = THREE_BYTE_QUALIFIERS.encode(16777215);
+        byte[] arr2 = new byte[] {-128, arr1[0], arr1[1], arr1[2]};
+        assertEquals(16777215, THREE_BYTE_QUALIFIERS.decode(arr2, 1, 3));
+        try {
+            THREE_BYTE_QUALIFIERS.decode(arr2, 0, 2);
+            fail();
+        } catch (InvalidQualifierBytesException expected) {}
+    }
+    
+    @Test
+    public void testFourByteQualifierEncodeDecode() {
+        assertEquals(1, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(1)));
+        assertEquals(127, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(127)));
+        assertEquals(63, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(63)));
+        assertEquals(130, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(130)));
+        assertEquals(128, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(128)));
+        assertEquals(129, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(129)));
+        assertEquals(32767, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(32767)));
+        assertEquals(32768, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(32768)));
+        assertEquals(65535, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(65535)));
+        assertEquals(Integer.MAX_VALUE, FOUR_BYTE_QUALIFIERS.decode(FOUR_BYTE_QUALIFIERS.encode(Integer.MAX_VALUE)));
+        byte[] arr1 = FOUR_BYTE_QUALIFIERS.encode(Integer.MAX_VALUE);
+        byte[] arr2 = new byte[] {-128, arr1[0], arr1[1], arr1[2], arr1[3]};
+        assertEquals(Integer.MAX_VALUE, FOUR_BYTE_QUALIFIERS.decode(arr2, 1, 4));
+        try {
+            FOUR_BYTE_QUALIFIERS.decode(arr2);
+            fail();
+        } catch (InvalidQualifierBytesException expected) {}
+        try {
+            FOUR_BYTE_QUALIFIERS.decode(arr2, 0, 3);
+            fail();
+        } catch (InvalidQualifierBytesException expected) {}
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 5feedb1..c427c8d 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
@@ -74,6 +74,7 @@ import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
+import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.expression.AndExpression;
 import org.apache.phoenix.expression.ByteBasedLikeExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
@@ -90,6 +91,7 @@ import org.apache.phoenix.expression.function.SingleAggregateFunction;
 import org.apache.phoenix.expression.function.SubstrFunction;
 import org.apache.phoenix.expression.function.SumAggregateFunction;
 import org.apache.phoenix.filter.MultiCQKeyValueComparisonFilter;
+import org.apache.phoenix.filter.MultiEncodedCQKeyValueComparisonFilter;
 import org.apache.phoenix.filter.MultiKeyValueComparisonFilter;
 import org.apache.phoenix.filter.RowKeyComparisonFilter;
 import org.apache.phoenix.filter.SingleCQKeyValueComparisonFilter;
@@ -108,6 +110,8 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PLongColumn;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
@@ -359,6 +363,10 @@ public class TestUtil {
     public static MultiKeyValueComparisonFilter multiKVFilter(Expression e) {
         return  new MultiCQKeyValueComparisonFilter(e);
     }
+    
+    public static MultiEncodedCQKeyValueComparisonFilter multiEncodedKVFilter(Expression e, QualifierEncodingScheme encodingScheme) {
+        return  new MultiEncodedCQKeyValueComparisonFilter(e, encodingScheme);
+    }
 
     public static Expression and(Expression... expressions) {
         return new AndExpression(Arrays.asList(expressions));
@@ -610,7 +618,7 @@ public class TestUtil {
     }
 
     public static void analyzeTable(Connection conn, String tableName) throws IOException, SQLException {
-    	analyzeTable(conn, tableName, false);
+        analyzeTable(conn, tableName, false);
     }
     
     public static void analyzeTable(Connection conn, String tableName, boolean transactional) throws IOException, SQLException {
@@ -652,17 +660,17 @@ public class TestUtil {
         Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * MILLIS_IN_DAY);
         stmt.setDate(6, date);
     }
-	
+    
     public static void validateRowKeyColumns(ResultSet rs, int i) throws SQLException {
-		assertTrue(rs.next());
-		assertEquals(rs.getString(1), "varchar" + String.valueOf(i));
-		assertEquals(rs.getString(2), "char" + String.valueOf(i));
-		assertEquals(rs.getInt(3), i);
-		assertEquals(rs.getInt(4), i);
-		assertEquals(rs.getBigDecimal(5), new BigDecimal(i*0.5d));
-		Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * MILLIS_IN_DAY);
-		assertEquals(rs.getDate(6), date);
-	}
+        assertTrue(rs.next());
+        assertEquals(rs.getString(1), "varchar" + String.valueOf(i));
+        assertEquals(rs.getString(2), "char" + String.valueOf(i));
+        assertEquals(rs.getInt(3), i);
+        assertEquals(rs.getInt(4), i);
+        assertEquals(rs.getBigDecimal(5), new BigDecimal(i*0.5d));
+        Date date = new Date(DateUtil.parseDate("2015-01-01 00:00:00").getTime() + (i - 1) * MILLIS_IN_DAY);
+        assertEquals(rs.getDate(6), date);
+    }
     
     public static String getTableName(Boolean mutable, Boolean transactional) {
         StringBuilder tableNameBuilder = new StringBuilder(DEFAULT_DATA_TABLE_NAME);
@@ -694,7 +702,7 @@ public class TestUtil {
                 
                 @Override
                 public SortOrder getSortOrder() {
-                	return SortOrder.getDefault();
+                    return SortOrder.getDefault();
                 }
                 
                 @Override
@@ -720,10 +728,14 @@ public class TestUtil {
                 public boolean isRowTimestamp() {
                     return false;
                 }
-    			@Override
-    			public boolean isDynamic() {
-    				return false;
-    			}
+                @Override
+                public boolean isDynamic() {
+                    return false;
+                }
+                @Override
+                public byte[] getColumnQualifierBytes() {
+                    return SINGLE_COLUMN_NAME.getBytes();
+                }
             })), null);
             aggregationManager.setAggregators(new ClientAggregators(Collections.<SingleAggregateFunction>singletonList(func), 1));
             ClientAggregators aggregators = aggregationManager.getAggregators();
@@ -765,15 +777,26 @@ public class TestUtil {
     
         // We simply write a marker row, request a major compaction, and then wait until the marker
         // row is gone
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        PTable table = pconn.getTable(new PTableKey(pconn.getTenantId(), tableName));
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
-        try (HTableInterface htable = services.getTable(Bytes.toBytes(tableName))) {
+        MutationState mutationState = pconn.getMutationState();
+        if (table.isTransactional()) {
+            mutationState.startTransaction();
+        }
+        try (HTableInterface htable = mutationState.getHTable(table)) {
             byte[] markerRowKey = Bytes.toBytes("TO_DELETE");
-        
+           
             Put put = new Put(markerRowKey);
-            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, HConstants.EMPTY_BYTE_ARRAY,
-                    HConstants.EMPTY_BYTE_ARRAY);
+            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_VALUE_BYTES, QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
             htable.put(put);
-            htable.delete(new Delete(markerRowKey));
+            Delete delete = new Delete(markerRowKey);
+            delete.deleteColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+            htable.delete(delete);
+            htable.close();
+            if (table.isTransactional()) {
+                mutationState.commit();
+            }
         
             HBaseAdmin hbaseAdmin = services.getAdmin();
             hbaseAdmin.flush(tableName);
@@ -782,19 +805,28 @@ public class TestUtil {
         
             boolean compactionDone = false;
             while (!compactionDone) {
-                Thread.sleep(2000L);
+                Thread.sleep(6000L);
                 Scan scan = new Scan();
                 scan.setStartRow(markerRowKey);
                 scan.setStopRow(Bytes.add(markerRowKey, new byte[] { 0 }));
                 scan.setRaw(true);
         
-                ResultScanner scanner = htable.getScanner(scan);
-                List<Result> results = Lists.newArrayList(scanner);
-                LOG.info("Results: " + results);
-                compactionDone = results.isEmpty();
-                scanner.close();
-        
+                try (HTableInterface htableForRawScan = services.getTable(Bytes.toBytes(tableName))) {
+                    ResultScanner scanner = htableForRawScan.getScanner(scan);
+                    List<Result> results = Lists.newArrayList(scanner);
+                    LOG.info("Results: " + results);
+                    compactionDone = results.isEmpty();
+                    scanner.close();
+                }
                 LOG.info("Compaction done: " + compactionDone);
+                
+                // need to run compaction after the next txn snapshot has been written so that compaction can remove deleted rows
+                if (!compactionDone && table.isTransactional()) {
+                    hbaseAdmin = services.getAdmin();
+                    hbaseAdmin.flush(tableName);
+                    hbaseAdmin.majorCompact(tableName);
+                    hbaseAdmin.close();
+                }
             }
         }
     }
@@ -821,4 +853,3 @@ public class TestUtil {
     }
 
 }
-

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-protocol/src/main/PTable.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/PTable.proto b/phoenix-protocol/src/main/PTable.proto
index a16263f..f2eb46c 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 bytes columnQualifierBytes = 15;
 }
 
 message PTableStats {
@@ -95,4 +96,12 @@ message PTable {
   optional string autoParititonSeqName = 31;
   optional bool isAppendOnlySchema = 32;
   optional bytes parentNameBytes = 33;
+  optional bytes storageScheme = 34;
+  optional bytes encodingScheme = 35;
+  repeated EncodedCQCounter encodedCQCounters = 36;
+}
+
+message EncodedCQCounter {
+	required string colFamily = 1;
+	required int32 counter = 2;
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-protocol/src/main/ServerCachingService.proto
----------------------------------------------------------------------
diff --git a/phoenix-protocol/src/main/ServerCachingService.proto b/phoenix-protocol/src/main/ServerCachingService.proto
index a45b18f..044c111 100644
--- a/phoenix-protocol/src/main/ServerCachingService.proto
+++ b/phoenix-protocol/src/main/ServerCachingService.proto
@@ -30,12 +30,47 @@ message ImmutableBytesWritable {
   required int32 length = 3;
 }
 
+message ColumnReference {
+  required bytes family = 1;
+  required bytes qualifier = 2;
+}
+
+message ColumnInfo {
+  optional string familyName = 1;
+  required string columnName = 2;
+}
+
+message IndexMaintainer {
+  required int32 saltBuckets = 1;
+  required bool isMultiTenant = 2;
+  optional bytes viewIndexId = 3;
+  repeated ColumnReference indexedColumns = 4;
+  repeated int32 indexedColumnTypeOrdinal = 5;
+  repeated ColumnReference dataTableColRefForCoveredColumns = 6;
+  repeated ColumnReference indexTableColRefForCoveredColumns = 7;
+  required bool isLocalIndex = 8;
+  required bytes indexTableName = 9;
+  required bool rowKeyOrderOptimizable = 10;
+  required bytes dataTableEmptyKeyValueColFamily = 11;
+  required ImmutableBytesWritable emptyKeyValueColFamily = 12;
+  optional bytes indexedExpressions = 13;
+  required bytes rowKeyMetadata = 14;
+  required int32 numDataTableColFamilies = 15;
+  required bool indexWalDisabled = 16;
+  required int32 indexRowKeyByteSize = 17;
+  required bool immutable = 18;
+  repeated ColumnInfo indexedColumnInfo = 19;
+  required int32 encodingScheme = 20;
+  required int32 immutableStorageScheme = 21;
+}
+
 message AddServerCacheRequest {
   optional bytes tenantId = 1;
   required bytes cacheId  = 2;
   required ImmutableBytesWritable cachePtr = 3;
   required ServerCacheFactory cacheFactory = 4;
   optional bytes txState = 5;
+  optional bool hasProtoBufIndexMaintainer = 6;
 }
 
 message AddServerCacheResponse {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 301aa7a..281fbe0 100644
--- a/pom.xml
+++ b/pom.xml
@@ -299,7 +299,7 @@
                 <forkCount>${numForkedIT}</forkCount>
                 <runOrder>alphabetical</runOrder>
                 <reuseForks>true</reuseForks>
-                <argLine>-enableassertions -Xmx2000m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
+                <argLine>-enableassertions -Xmx2500m -XX:MaxPermSize=128m -Djava.security.egd=file:/dev/./urandom "-Djava.library.path=${hadoop.library.path}${path.separator}${java.library.path}" -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=./target/</argLine>
                 <redirectTestOutputToFile>${test.output.tofile}</redirectTestOutputToFile>
                 <testSourceDirectory>${basedir}/src/it/java</testSourceDirectory>
                 <groups>org.apache.phoenix.end2end.ClientManagedTimeTest</groups>


[10/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingProtocol.java
index b201c8e..139a69c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingProtocol.java
@@ -36,7 +36,7 @@ import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
  */
 public interface ServerCachingProtocol {
     public static interface ServerCacheFactory extends Writable {
-        public Closeable newCache(ImmutableBytesWritable cachePtr, byte[] txState, MemoryChunk chunk) throws SQLException;
+        public Closeable newCache(ImmutableBytesWritable cachePtr, byte[] txState, MemoryChunk chunk, boolean useProtoForIndexMaintainer) throws SQLException;
     }
     /**
      * Add the cache to the region server cache.  

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 db3c792..c5854d3 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
@@ -101,7 +101,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;
@@ -109,6 +112,7 @@ import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PLong;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.LogUtil;
@@ -181,9 +185,9 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     }
 
     private void commitBatch(Region region, List<Mutation> mutations, byte[] indexUUID, long blockingMemstoreSize,
-            byte[] indexMaintainersPtr, byte[] txState) throws IOException {
+            byte[] indexMaintainersPtr, byte[] txState, boolean useIndexProto) throws IOException {
         if (indexMaintainersPtr != null) {
-            mutations.get(0).setAttribute(PhoenixIndexCodec.INDEX_MD, indexMaintainersPtr);
+            mutations.get(0).setAttribute(useIndexProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMaintainersPtr);
         }
 
         if (txState != null) {
@@ -213,13 +217,13 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     }
     
     private void commitBatchWithHTable(HTable table, Region region, List<Mutation> mutations, byte[] indexUUID,
-            long blockingMemstoreSize, byte[] indexMaintainersPtr, byte[] txState) throws IOException {
+            long blockingMemstoreSize, byte[] indexMaintainersPtr, byte[] txState, boolean useIndexProto) throws IOException {
 
         if (indexUUID != null) {
             // Need to add indexMaintainers for each mutation as table.batch can be distributed across servers
             for (Mutation m : mutations) {
                 if (indexMaintainersPtr != null) {
-                    m.setAttribute(PhoenixIndexCodec.INDEX_MD, indexMaintainersPtr);
+                    m.setAttribute(useIndexProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMaintainersPtr);
                 }
                 if (txState != null) {
                     m.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
@@ -328,8 +332,13 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             }
             values = new byte[projectedTable.getPKColumns().size()][];
         }
-        byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
-        List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
+        boolean useProto = false;
+        byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD_PROTO);
+        useProto = localIndexBytes != null;
+        if (localIndexBytes == null) {
+            localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
+        }
+        List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes, useProto);
         List<Mutation> indexMutations = localIndexBytes == null ? Collections.<Mutation>emptyList() : Lists.<Mutation>newArrayListWithExpectedSize(1024);
         
         RegionScanner theScanner = s;
@@ -370,6 +379,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         ColumnReference[] dataColumns = IndexUtil.deserializeDataTableColumnsToJoin(scan);
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        boolean useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan));
         if ((localIndexScan && !isDelete && !isDescRowKeyOrderUpgrade) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
@@ -378,11 +388,11 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
             ImmutableBytesWritable tempPtr = new ImmutableBytesWritable();
             theScanner =
                     getWrappedScanner(c, theScanner, offset, scan, dataColumns, tupleProjector, 
-                        region, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr);
+                        region, indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr, useQualifierAsIndex);
         } 
         
         if (j != null)  {
-            theScanner = new HashJoinRegionScanner(theScanner, p, j, ScanUtil.getTenantId(scan), env);
+            theScanner = new HashJoinRegionScanner(theScanner, p, j, ScanUtil.getTenantId(scan), env, useQualifierAsIndex, useNewValueColumnQualifier);
         }
         
         int batchSize = 0;
@@ -421,13 +431,20 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
         Aggregator[] rowAggregators = aggregators.getAggregators();
         boolean hasMore;
         boolean hasAny = false;
-        MultiKeyValueTuple result = new MultiKeyValueTuple();
+        Pair<Integer, Integer> minMaxQualifiers = EncodedColumnsUtil.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)));
         }
         long rowCount = 0;
         final RegionScanner innerScanner = theScanner;
-        byte[] indexMaintainersPtr = scan.getAttribute(PhoenixIndexCodec.INDEX_MD);
+        boolean useIndexProto = true;
+        byte[] indexMaintainersPtr = scan.getAttribute(PhoenixIndexCodec.INDEX_PROTO_MD);
+        // for backward compatiblity fall back to look by the old attribute
+        if (indexMaintainersPtr == null) {
+            indexMaintainersPtr = scan.getAttribute(PhoenixIndexCodec.INDEX_MD);
+            useIndexProto = false;
+        }
         boolean acquiredLock = false;
         try {
             if(needToWrite) {
@@ -439,7 +456,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(), encodingScheme) : 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
@@ -654,7 +671,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                                 MutationState.getMutationBatchList(batchSize, batchSizeBytes, mutations);
                             for (List<Mutation> batchMutations : batchMutationList) {
                                 commit(region, batchMutations, indexUUID, blockingMemStoreSize, indexMaintainersPtr,
-                                        txState, areMutationInSameRegion, targetHTable);
+                                        txState, areMutationInSameRegion, targetHTable, useIndexProto);
                                 batchMutations.clear();
                             }
                             mutations.clear();
@@ -662,7 +679,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                             List<List<Mutation>> batchIndexMutationList =
                                 MutationState.getMutationBatchList(batchSize, batchSizeBytes, indexMutations);
                             for (List<Mutation> batchIndexMutations : batchIndexMutationList) {
-                                commitBatch(region, batchIndexMutations, null, blockingMemStoreSize, null, txState);
+                                commitBatch(region, batchIndexMutations, null, blockingMemStoreSize, null, txState, useIndexProto);
                                 batchIndexMutations.clear();
                             }
                             indexMutations.clear();
@@ -673,12 +690,12 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                 } while (hasMore);
                 if (!mutations.isEmpty()) {
                     commit(region, mutations, indexUUID, blockingMemStoreSize, indexMaintainersPtr, txState,
-                            areMutationInSameRegion, targetHTable);
+                            areMutationInSameRegion, targetHTable, useIndexProto);
                     mutations.clear();
                 }
 
                 if (!indexMutations.isEmpty()) {
-                    commitBatch(region, indexMutations, null, blockingMemStoreSize, indexMaintainersPtr, txState);
+                    commitBatch(region, indexMutations, null, blockingMemStoreSize, indexMaintainersPtr, txState, useIndexProto);
                     indexMutations.clear();
                 }
             }
@@ -735,14 +752,14 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     }
 
     private void commit(Region region, List<Mutation> mutations, byte[] indexUUID, long blockingMemstoreSize,
-            byte[] indexMaintainersPtr, byte[] txState, boolean areMutationsForSameRegion, HTable hTable)
+            byte[] indexMaintainersPtr, byte[] txState, boolean areMutationsForSameRegion, HTable hTable, boolean useIndexProto)
             throws IOException {
         if (!areMutationsForSameRegion) {
             assert hTable != null;// table cannot be null
             commitBatchWithHTable(hTable, region, mutations, indexUUID, blockingMemstoreSize, indexMaintainersPtr,
-                    txState);
+                    txState, useIndexProto);
         } else {
-            commitBatch(region, mutations, indexUUID, blockingMemstoreSize, indexMaintainersPtr, txState);
+            commitBatch(region, mutations, indexUUID, blockingMemstoreSize, indexMaintainersPtr, txState, useIndexProto);
         }
     }
 
@@ -804,7 +821,13 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
     
     private RegionScanner rebuildIndices(final RegionScanner innerScanner, final Region region, final Scan scan,
             Configuration config) throws IOException {
-        byte[] indexMetaData = scan.getAttribute(PhoenixIndexCodec.INDEX_MD);
+        byte[] indexMetaData = scan.getAttribute(PhoenixIndexCodec.INDEX_PROTO_MD);
+        boolean useProto = true;
+        // for backward compatibility fall back to look up by the old attribute
+        if (indexMetaData == null) {
+            useProto = false;
+            indexMetaData = scan.getAttribute(PhoenixIndexCodec.INDEX_MD);
+        }
         boolean hasMore;
         long rowCount = 0;
         try {
@@ -824,7 +847,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                             if (KeyValue.Type.codeToType(cell.getTypeByte()) == KeyValue.Type.Put) {
                                 if (put == null) {
                                     put = new Put(CellUtil.cloneRow(cell));
-                                    put.setAttribute(PhoenixIndexCodec.INDEX_MD, indexMetaData);
+                                    put.setAttribute(useProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMetaData);
                                     put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                                     put.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS,
                                             PDataType.TRUE_BYTES);
@@ -834,7 +857,7 @@ public class UngroupedAggregateRegionObserver extends BaseScannerRegionObserver
                             } else {
                                 if (del == null) {
                                     del = new Delete(CellUtil.cloneRow(cell));
-                                    del.setAttribute(PhoenixIndexCodec.INDEX_MD, indexMetaData);
+                                    del.setAttribute(useProto ? PhoenixIndexCodec.INDEX_PROTO_MD : PhoenixIndexCodec.INDEX_MD, indexMetaData);
                                     del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
                                     del.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS,
                                             PDataType.TRUE_BYTES);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..b7ca46f 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 bytes columnQualifierBytes = 15;
+    /**
+     * <code>optional bytes columnQualifierBytes = 15;</code>
+     */
+    boolean hasColumnQualifierBytes();
+    /**
+     * <code>optional bytes columnQualifierBytes = 15;</code>
+     */
+    com.google.protobuf.ByteString getColumnQualifierBytes();
   }
   /**
    * Protobuf type {@code PColumn}
@@ -391,6 +401,11 @@ public final class PTableProtos {
               isDynamic_ = input.readBool();
               break;
             }
+            case 122: {
+              bitField0_ |= 0x00004000;
+              columnQualifierBytes_ = input.readBytes();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -709,6 +724,22 @@ public final class PTableProtos {
       return isDynamic_;
     }
 
+    // optional bytes columnQualifierBytes = 15;
+    public static final int COLUMNQUALIFIERBYTES_FIELD_NUMBER = 15;
+    private com.google.protobuf.ByteString columnQualifierBytes_;
+    /**
+     * <code>optional bytes columnQualifierBytes = 15;</code>
+     */
+    public boolean hasColumnQualifierBytes() {
+      return ((bitField0_ & 0x00004000) == 0x00004000);
+    }
+    /**
+     * <code>optional bytes columnQualifierBytes = 15;</code>
+     */
+    public com.google.protobuf.ByteString getColumnQualifierBytes() {
+      return columnQualifierBytes_;
+    }
+
     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;
+      columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
     }
     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.writeBytes(15, columnQualifierBytes_);
+      }
       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
+          .computeBytesSize(15, columnQualifierBytes_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -957,6 +996,11 @@ public final class PTableProtos {
         result = result && (getIsDynamic()
             == other.getIsDynamic());
       }
+      result = result && (hasColumnQualifierBytes() == other.hasColumnQualifierBytes());
+      if (hasColumnQualifierBytes()) {
+        result = result && getColumnQualifierBytes()
+            .equals(other.getColumnQualifierBytes());
+      }
       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 (hasColumnQualifierBytes()) {
+        hash = (37 * hash) + COLUMNQUALIFIERBYTES_FIELD_NUMBER;
+        hash = (53 * hash) + getColumnQualifierBytes().hashCode();
+      }
       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);
+        columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
+        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.columnQualifierBytes_ = columnQualifierBytes_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -1309,6 +1363,9 @@ public final class PTableProtos {
         if (other.hasIsDynamic()) {
           setIsDynamic(other.getIsDynamic());
         }
+        if (other.hasColumnQualifierBytes()) {
+          setColumnQualifierBytes(other.getColumnQualifierBytes());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -1909,6 +1966,42 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional bytes columnQualifierBytes = 15;
+      private com.google.protobuf.ByteString columnQualifierBytes_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes columnQualifierBytes = 15;</code>
+       */
+      public boolean hasColumnQualifierBytes() {
+        return ((bitField0_ & 0x00004000) == 0x00004000);
+      }
+      /**
+       * <code>optional bytes columnQualifierBytes = 15;</code>
+       */
+      public com.google.protobuf.ByteString getColumnQualifierBytes() {
+        return columnQualifierBytes_;
+      }
+      /**
+       * <code>optional bytes columnQualifierBytes = 15;</code>
+       */
+      public Builder setColumnQualifierBytes(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00004000;
+        columnQualifierBytes_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes columnQualifierBytes = 15;</code>
+       */
+      public Builder clearColumnQualifierBytes() {
+        bitField0_ = (bitField0_ & ~0x00004000);
+        columnQualifierBytes_ = getDefaultInstance().getColumnQualifierBytes();
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PColumn)
     }
 
@@ -3372,6 +3465,51 @@ 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();
+
+    // optional bytes encodingScheme = 35;
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    boolean hasEncodingScheme();
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    com.google.protobuf.ByteString getEncodingScheme();
+
+    // repeated .EncodedCQCounter encodedCQCounters = 36;
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> 
+        getEncodedCQCountersList();
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index);
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    int getEncodedCQCountersCount();
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
+        getEncodedCQCountersOrBuilderList();
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder getEncodedCQCountersOrBuilder(
+        int index);
   }
   /**
    * Protobuf type {@code PTable}
@@ -3407,6 +3545,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 +3738,24 @@ public final class PTableProtos {
               parentNameBytes_ = input.readBytes();
               break;
             }
+            case 274: {
+              bitField0_ |= 0x20000000;
+              storageScheme_ = input.readBytes();
+              break;
+            }
+            case 282: {
+              bitField0_ |= 0x40000000;
+              encodingScheme_ = input.readBytes();
+              break;
+            }
+            case 290: {
+              if (!((mutable_bitField1_ & 0x00000004) == 0x00000004)) {
+                encodedCQCounters_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter>();
+                mutable_bitField1_ |= 0x00000004;
+              }
+              encodedCQCounters_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.PARSER, extensionRegistry));
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3616,6 +3773,9 @@ public final class PTableProtos {
         if (((mutable_bitField0_ & 0x00020000) == 0x00020000)) {
           physicalNames_ = java.util.Collections.unmodifiableList(physicalNames_);
         }
+        if (((mutable_bitField1_ & 0x00000004) == 0x00000004)) {
+          encodedCQCounters_ = java.util.Collections.unmodifiableList(encodedCQCounters_);
+        }
         this.unknownFields = unknownFields.build();
         makeExtensionsImmutable();
       }
@@ -4275,6 +4435,74 @@ 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_;
+    }
+
+    // optional bytes encodingScheme = 35;
+    public static final int ENCODINGSCHEME_FIELD_NUMBER = 35;
+    private com.google.protobuf.ByteString encodingScheme_;
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    public boolean hasEncodingScheme() {
+      return ((bitField0_ & 0x40000000) == 0x40000000);
+    }
+    /**
+     * <code>optional bytes encodingScheme = 35;</code>
+     */
+    public com.google.protobuf.ByteString getEncodingScheme() {
+      return encodingScheme_;
+    }
+
+    // repeated .EncodedCQCounter encodedCQCounters = 36;
+    public static final int ENCODEDCQCOUNTERS_FIELD_NUMBER = 36;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> encodedCQCounters_;
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> getEncodedCQCountersList() {
+      return encodedCQCounters_;
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounterOrBuilder> 
+        getEncodedCQCountersOrBuilderList() {
+      return encodedCQCounters_;
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    public int getEncodedCQCountersCount() {
+      return encodedCQCounters_.size();
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter getEncodedCQCounters(int index) {
+      return encodedCQCounters_.get(index);
+    }
+    /**
+     * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</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 +4536,9 @@ public final class PTableProtos {
       autoParititonSeqName_ = "";
       isAppendOnlySchema_ = false;
       parentNameBytes_ = com.google.protobuf.ByteString.EMPTY;
+      storageScheme_ = com.google.protobuf.ByteString.EMPTY;
+      encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
+      encodedCQCounters_ = java.util.Collections.emptyList();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4362,6 +4593,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 +4702,15 @@ public final class PTableProtos {
       if (((bitField0_ & 0x10000000) == 0x10000000)) {
         output.writeBytes(33, parentNameBytes_);
       }
+      if (((bitField0_ & 0x20000000) == 0x20000000)) {
+        output.writeBytes(34, storageScheme_);
+      }
+      if (((bitField0_ & 0x40000000) == 0x40000000)) {
+        output.writeBytes(35, encodingScheme_);
+      }
+      for (int i = 0; i < encodedCQCounters_.size(); i++) {
+        output.writeMessage(36, encodedCQCounters_.get(i));
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4607,6 +4853,18 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBytesSize(33, parentNameBytes_);
       }
+      if (((bitField0_ & 0x20000000) == 0x20000000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(34, storageScheme_);
+      }
+      if (((bitField0_ & 0x40000000) == 0x40000000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(35, encodingScheme_);
+      }
+      for (int i = 0; i < encodedCQCounters_.size(); i++) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(36, encodedCQCounters_.get(i));
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4781,6 +5039,18 @@ 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 && (hasEncodingScheme() == other.hasEncodingScheme());
+      if (hasEncodingScheme()) {
+        result = result && getEncodingScheme()
+            .equals(other.getEncodingScheme());
+      }
+      result = result && getEncodedCQCountersList()
+          .equals(other.getEncodedCQCountersList());
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4922,6 +5192,18 @@ 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 (hasEncodingScheme()) {
+        hash = (37 * hash) + ENCODINGSCHEME_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodingScheme().hashCode();
+      }
+      if (getEncodedCQCountersCount() > 0) {
+        hash = (37 * hash) + ENCODEDCQCOUNTERS_FIELD_NUMBER;
+        hash = (53 * hash) + getEncodedCQCountersList().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5025,6 +5307,7 @@ public final class PTableProtos {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
           getColumnsFieldBuilder();
           getIndexesFieldBuilder();
+          getEncodedCQCountersFieldBuilder();
         }
       }
       private static Builder create() {
@@ -5105,6 +5388,16 @@ 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);
+        encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
+        bitField1_ = (bitField1_ & ~0x00000002);
+        if (encodedCQCountersBuilder_ == null) {
+          encodedCQCounters_ = java.util.Collections.emptyList();
+          bitField1_ = (bitField1_ & ~0x00000004);
+        } else {
+          encodedCQCountersBuilder_.clear();
+        }
         return this;
       }
 
@@ -5132,6 +5425,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 +5566,23 @@ public final class PTableProtos {
           to_bitField0_ |= 0x10000000;
         }
         result.parentNameBytes_ = parentNameBytes_;
+        if (((from_bitField1_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x20000000;
+        }
+        result.storageScheme_ = storageScheme_;
+        if (((from_bitField1_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x40000000;
+        }
+        result.encodingScheme_ = encodingScheme_;
+        if (encodedCQCountersBuilder_ == null) {
+          if (((bitField1_ & 0x00000004) == 0x00000004)) {
+            encodedCQCounters_ = java.util.Collections.unmodifiableList(encodedCQCounters_);
+            bitField1_ = (bitField1_ & ~0x00000004);
+          }
+          result.encodedCQCounters_ = encodedCQCounters_;
+        } else {
+          result.encodedCQCounters_ = encodedCQCountersBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -5441,6 +5752,38 @@ public final class PTableProtos {
         if (other.hasParentNameBytes()) {
           setParentNameBytes(other.getParentNameBytes());
         }
+        if (other.hasStorageScheme()) {
+          setStorageScheme(other.getStorageScheme());
+        }
+        if (other.hasEncodingScheme()) {
+          setEncodingScheme(other.getEncodingScheme());
+        }
+        if (encodedCQCountersBuilder_ == null) {
+          if (!other.encodedCQCounters_.isEmpty()) {
+            if (encodedCQCounters_.isEmpty()) {
+              encodedCQCounters_ = other.encodedCQCounters_;
+              bitField1_ = (bitField1_ & ~0x00000004);
+            } else {
+              ensureEncodedCQCountersIsMutable();
+              encodedCQCounters_.addAll(other.encodedCQCounters_);
+            }
+            onChanged();
+          }
+        } else {
+          if (!other.encodedCQCounters_.isEmpty()) {
+            if (encodedCQCountersBuilder_.isEmpty()) {
+              encodedCQCountersBuilder_.dispose();
+              encodedCQCountersBuilder_ = null;
+              encodedCQCounters_ = other.encodedCQCounters_;
+              bitField1_ = (bitField1_ & ~0x00000004);
+              encodedCQCountersBuilder_ = 
+                com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ?
+                   getEncodedCQCountersFieldBuilder() : null;
+            } else {
+              encodedCQCountersBuilder_.addAllMessages(other.encodedCQCounters_);
+            }
+          }
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -5494,6 +5837,12 @@ public final class PTableProtos {
             return false;
           }
         }
+        for (int i = 0; i < getEncodedCQCountersCount(); i++) {
+          if (!getEncodedCQCounters(i).isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -5515,6 +5864,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 +7518,1043 @@ public final class PTableProtos {
         return this;
       }
 
-      // @@protoc_insertion_point(builder_scope:PTable)
-    }
-
-    static {
-      defaultInstance = new PTable(true);
-      defaultInstance.initFields();
-    }
+      // 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;
+      }
 
-    // @@protoc_insertion_point(class_scope:PTable)
+      // optional bytes encodingScheme = 35;
+      private com.google.protobuf.ByteString encodingScheme_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public boolean hasEncodingScheme() {
+        return ((bitField1_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public com.google.protobuf.ByteString getEncodingScheme() {
+        return encodingScheme_;
+      }
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public Builder setEncodingScheme(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
   }
+  bitField1_ |= 0x00000002;
+        encodingScheme_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bytes encodingScheme = 35;</code>
+       */
+      public Builder clearEncodingScheme() {
+        bitField1_ = (bitField1_ & ~0x00000002);
+        encodingScheme_ = getDefaultInstance().getEncodingScheme();
+        onChanged();
+        return this;
+      }
 
-  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 = 36;
+      private java.util.List<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter> encodedCQCounters_ =
+        java.util.Collections.emptyList();
+      private void ensureEncodedCQCountersIsMutable() {
+        if (!((bitField1_ & 0x00000004) == 0x00000004)) {
+          encodedCQCounters_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter>(encodedCQCounters_);
+          bitField1_ |= 0x00000004;
+         }
+      }
 
-  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 = 36;</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 = 36;</code>
+       */
+      public int getEncodedCQCountersCount() {
+        if (encodedCQCountersBuilder_ == null) {
+          return encodedCQCounters_.size();
+        } else {
+          return encodedCQCountersBuilder_.getCount();
+        }
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</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 = 36;</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 = 36;</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 = 36;</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 = 36;</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 = 36;</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 = 36;</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 = 36;</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 = 36;</code>
+       */
+      public Builder clearEncodedCQCounters() {
+        if (encodedCQCountersBuilder_ == null) {
+          encodedCQCounters_ = java.util.Collections.emptyList();
+          bitField1_ = (bitField1_ & ~0x00000004);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.clear();
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+       */
+      public Builder removeEncodedCQCounters(int index) {
+        if (encodedCQCountersBuilder_ == null) {
+          ensureEncodedCQCountersIsMutable();
+          encodedCQCounters_.remove(index);
+          onChanged();
+        } else {
+          encodedCQCountersBuilder_.remove(index);
+        }
+        return this;
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder getEncodedCQCountersBuilder(
+          int index) {
+        return getEncodedCQCountersFieldBuilder().getBuilder(index);
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</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 = 36;</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 = 36;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder addEncodedCQCountersBuilder() {
+        return getEncodedCQCountersFieldBuilder().addBuilder(
+            org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.getDefaultInstance());
+      }
+      /**
+       * <code>repeated .EncodedCQCounter encodedCQCounters = 36;</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 = 36;</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_ & 0x00000004) == 0x00000004),
+                  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\"\304\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\034\n\024columnQualifierBytes\030\017 \001(\014" +
+      "\"\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\rke" +
+      "yBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030\005 \001" +
+      "(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts\"\354\006" +
+      "\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016tab" +
+      "leNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162\013.P" +
+      "TableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequenc" +
+      "eNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013pkNa" +
+      "meBytes\030\007 \001(\014\022\021\n\tbucketNum\030\010 \002(\005\022\031\n\007colu" +
+      "mns\030\t \003(\0132\010.PColumn\022\030\n\007indexes\030\n \003(\0132\007.P" +
+      "Table\022\027\n\017isImmutableRows\030\013 \002(\010\022\032\n\022dataTa",
+      "bleNameBytes\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\rviewStatemen" +
+      "t\030\022 \001(\014\022\025\n\rphysicalNames\030\023 \003(\014\022\020\n\010tenant" +
+      "Id\030\024 \001(\014\022\023\n\013viewIndexId\030\025 \001(\005\022\021\n\tindexTy" +
+      "pe\030\026 \001(\014\022\026\n\016statsTimeStamp\030\027 \001(\003\022\022\n\nstor" +
+      "eNulls\030\030 \001(\010\022\027\n\017baseColumnCount\030\031 \001(\005\022\036\n" +
+      "\026rowKeyOrderOptimizable\030\032 \001(\010\022\025\n\rtransac" +
+      "tional\030\033 \001(\010\022\034\n\024updateCacheFrequency\030\034 \001" +
+      "(\003\022\035\n\025indexDisableTimestamp\030\035 \001(\003\022\031\n\021isN",
+      "amespaceMapped\030\036 \001(\010\022\034\n\024autoParititonSeq" +
+      "Name\030\037 \001(\t\022\032\n\022isAppendOnlySchema\030  \001(\010\022\027" +
+      "\n\017parentNameBytes\030! \001(\014\022\025\n\rstorageScheme" +
+      "\030\" \001(\014\022\026\n\016encodingScheme\030# \001(\014\022,\n\021encode" +
+      "dCQCounters\030$ \003(\0132\021.EncodedCQCounter\"6\n\020" +
+      "EncodedCQCounter\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007c" +
+      "ounter\030\002 \002(\005*A\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010" +
+      "\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B" +
+      "@\n(org.apache.phoenix.coprocessor.genera" +
+      "tedB\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", "ColumnQualifierBytes", });
+          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", "EncodingScheme", "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;
         }
       };


[16/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 211145e..6b57148 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
@@ -54,22 +54,41 @@ import org.junit.runners.Parameterized.Parameters;
 public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     
     private final boolean isMultiTenant;
+    private final boolean columnEncoded;
     
     private final String TENANT_SPECIFIC_URL1 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant1";
     private final String TENANT_SPECIFIC_URL2 = getUrl() + ';' + TENANT_ID_ATTRIB + "=tenant2";
     
-    public AlterTableWithViewsIT(boolean isMultiTenant) {
+    public AlterTableWithViewsIT(boolean isMultiTenant, boolean columnEncoded) {
         this.isMultiTenant = isMultiTenant;
+        this.columnEncoded = columnEncoded;
     }
     
-    @Parameters(name="AlterTableWithViewsIT_multiTenant={0}") // name is used by failsafe as file name in reports
-    public static Collection<Boolean> data() {
-        return Arrays.asList(false, true);
+    @Parameters(name="AlterTableWithViewsIT_multiTenant={0}, columnEncoded={1}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] { 
+                { false, false }, { false, true },
+                { true, false }, { true, true } });
     }
-	
+    
     private String generateDDL(String format) {
+        return generateDDL("", format);
+    }
+    
+    private String generateDDL(String options, String format) {
+        StringBuilder optionsBuilder = new StringBuilder(options);
+        if (!columnEncoded) {
+            if (optionsBuilder.length()!=0)
+                optionsBuilder.append(",");
+            optionsBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
+        if (isMultiTenant) {
+            if (optionsBuilder.length()!=0)
+                optionsBuilder.append(",");
+            optionsBuilder.append("MULTI_TENANT=true");
+        }
         return String.format(format, isMultiTenant ? "TENANT_ID VARCHAR NOT NULL, " : "",
-            isMultiTenant ? "TENANT_ID, " : "", isMultiTenant ? "MULTI_TENANT=true" : "");
+            isMultiTenant ? "TENANT_ID, " : "", optionsBuilder.toString());
     }
     
     @Test
@@ -92,7 +111,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             
             // adding a new pk column and a new regular column
             conn.createStatement().execute("ALTER TABLE " + tableName + " ADD COL3 varchar(10) PRIMARY KEY, COL4 integer");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL3", "COL4");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 7, 5, "ID", "COL1", "COL2", "COL3", "COL4", "VIEW_COL1", "VIEW_COL2");
         } 
     }
@@ -109,28 +128,27 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                             + " COL1 integer NOT NULL,"
                             + " COL2 bigint NOT NULL,"
                             + " CONSTRAINT NAME_PK PRIMARY KEY (%s ID, COL1, COL2)"
-                            + " ) UPDATE_CACHE_FREQUENCY=15 "
-                            + (isMultiTenant ? ",%s" : "%s");
-            conn.createStatement().execute(generateDDL(ddlFormat));
+                            + " ) %s ";
+            conn.createStatement().execute(generateDDL("UPDATE_CACHE_FREQUENCY=2", ddlFormat));
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable1 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
             viewConn.createStatement().execute("CREATE VIEW " + viewOfTable2 + " ( VIEW_COL1 DECIMAL(10,2), VIEW_COL2 VARCHAR ) AS SELECT * FROM " + tableName);
-            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 5");
+            viewConn.createStatement().execute("ALTER VIEW " + viewOfTable2 + " SET UPDATE_CACHE_FREQUENCY = 1");
             
             PhoenixConnection phoenixConn = conn.unwrap(PhoenixConnection.class);
             PTable table = phoenixConn.getTable(new PTableKey(null, tableName));
             PName tenantId = isMultiTenant ? PNameFactory.newName("tenant1") : null;
             assertFalse(table.isImmutableRows());
-            assertEquals(15, table.getUpdateCacheFrequency());
+            assertEquals(2, table.getUpdateCacheFrequency());
             PTable viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
             assertFalse(viewTable1.isImmutableRows());
-            assertEquals(15, viewTable1.getUpdateCacheFrequency());
+            assertEquals(2, viewTable1.getUpdateCacheFrequency());
             // query the view to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2);
             PTable viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
             assertFalse(viewTable2.isImmutableRows());
-            assertEquals(5, viewTable2.getUpdateCacheFrequency());
+            assertEquals(1, viewTable2.getUpdateCacheFrequency());
             
-            conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=10");
+            conn.createStatement().execute("ALTER TABLE " + tableName + " SET IMMUTABLE_ROWS=true, UPDATE_CACHE_FREQUENCY=3");
             // query the views to force the table cache to be updated
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable1);
             viewConn.createStatement().execute("SELECT * FROM "+viewOfTable2);
@@ -138,16 +156,16 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
             phoenixConn = conn.unwrap(PhoenixConnection.class);
             table = phoenixConn.getTable(new PTableKey(null, tableName));
             assertTrue(table.isImmutableRows());
-            assertEquals(10, table.getUpdateCacheFrequency());
+            assertEquals(3, table.getUpdateCacheFrequency());
             
             viewTable1 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable1));
             assertTrue(viewTable1.isImmutableRows());
-            assertEquals(10, viewTable1.getUpdateCacheFrequency());
+            assertEquals(3, viewTable1.getUpdateCacheFrequency());
             
             viewTable2 = viewConn.unwrap(PhoenixConnection.class).getTable(new PTableKey(tenantId, viewOfTable2));
             assertTrue(viewTable2.isImmutableRows());
             // update cache frequency is not propagated to the view since it was altered on the view
-            assertEquals(5, viewTable2.getUpdateCacheFrequency());
+            assertEquals(1, viewTable2.getUpdateCacheFrequency());
         } 
     }
     
@@ -174,7 +192,7 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
 
             // drop two columns from the base table
             conn.createStatement().execute("ALTER TABLE " + tableName + " DROP COLUMN COL3, COL5");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 4,
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 4,
                 QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "COL4");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 6, 4,
                 "ID", "COL1", "COL2", "COL4", "VIEW_COL1", "VIEW_COL2");
@@ -253,38 +271,49 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
                 assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
             }
             
-            // validate that there were no columns added to the table or view
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
+            // validate that there were no columns added to the table or view, if its table is column encoded the sequence number changes when we increment the cq counter
+            assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 1 : 0, 3, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2");
             assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 0, 9, 3, "ID", "COL1", "COL2", "VIEW_COL1", "VIEW_COL2", "VIEW_COL3", "VIEW_COL4", "VIEW_COL5", "VIEW_COL6");
             
-            // should succeed 
-            conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
-            assertTableDefinition(conn, tableName, PTableType.TABLE, null, 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
-            assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
-            
-            // query table
-            ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
-            assertTrue(rs.next());
-            assertEquals("view1", rs.getString("ID"));
-            assertEquals(12, rs.getInt("COL1"));
-            assertEquals(13, rs.getInt("COL2"));
-            assertEquals("view5", rs.getString("VIEW_COL2"));
-            assertEquals(17, rs.getInt("VIEW_COL4"));
-            assertFalse(rs.next());
-
-            // query view
-            rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
-            assertTrue(rs.next());
-            assertEquals("view1", rs.getString("ID"));
-            assertEquals(12, rs.getInt("COL1"));
-            assertEquals(13, rs.getInt("COL2"));
-            assertEquals(14, rs.getInt("VIEW_COL1"));
-            assertEquals("view5", rs.getString("VIEW_COL2"));
-            assertEquals("view6", rs.getString("VIEW_COL3"));
-            assertEquals(17, rs.getInt("VIEW_COL4"));
-            assertEquals(18, rs.getInt("VIEW_COL5"));
-            assertEquals("view9", rs.getString("VIEW_COL6"));
-            assertFalse(rs.next());
+            if (columnEncoded) {
+                try {
+                    // adding a key value column to the base table that already exists in the view is not allowed
+                    conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+                    fail();
+                } catch (SQLException e) {
+                    assertEquals("Unexpected exception", CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+                }
+            }
+            else {
+                // should succeed 
+                conn.createStatement().execute("ALTER TABLE " + tableName + " ADD VIEW_COL4 DECIMAL, VIEW_COL2 VARCHAR(256)");
+                assertTableDefinition(conn, tableName, PTableType.TABLE, null, columnEncoded ? 2 : 1, 5, QueryConstants.BASE_TABLE_BASE_COLUMN_COUNT, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2");
+                assertTableDefinition(conn, viewOfTable, PTableType.VIEW, tableName, 1, 9, 5, "ID", "COL1", "COL2", "VIEW_COL4", "VIEW_COL2", "VIEW_COL1", "VIEW_COL3", "VIEW_COL5", "VIEW_COL6");
+            
+                // query table
+                ResultSet rs = stmt.executeQuery("SELECT * FROM " + tableName);
+                assertTrue(rs.next());
+                assertEquals("view1", rs.getString("ID"));
+                assertEquals(12, rs.getInt("COL1"));
+                assertEquals(13, rs.getInt("COL2"));
+                assertEquals("view5", rs.getString("VIEW_COL2"));
+                assertEquals(17, rs.getInt("VIEW_COL4"));
+                assertFalse(rs.next());
+    
+                // query view
+                rs = stmt.executeQuery("SELECT * FROM " + viewOfTable);
+                assertTrue(rs.next());
+                assertEquals("view1", rs.getString("ID"));
+                assertEquals(12, rs.getInt("COL1"));
+                assertEquals(13, rs.getInt("COL2"));
+                assertEquals(14, rs.getInt("VIEW_COL1"));
+                assertEquals("view5", rs.getString("VIEW_COL2"));
+                assertEquals("view6", rs.getString("VIEW_COL3"));
+                assertEquals(17, rs.getInt("VIEW_COL4"));
+                assertEquals(18, rs.getInt("VIEW_COL5"));
+                assertEquals("view9", rs.getString("VIEW_COL6"));
+                assertFalse(rs.next());
+            }
         } 
     }
     
@@ -603,9 +632,9 @@ public class AlterTableWithViewsIT extends ParallelStatsDisabledIT {
     
     @Test
     public void testAlteringViewThatHasChildViews() throws Exception {
-        String baseTable = "testAlteringViewThatHasChildViews";
-        String childView = "childView";
-        String grandChildView = "grandChildView";
+        String baseTable = generateUniqueName();
+        String childView = baseTable + "cildView";
+        String grandChildView = baseTable + "grandChildView";
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection viewConn = isMultiTenant ? DriverManager.getConnection(TENANT_SPECIFIC_URL1) : conn ) {
             String ddlFormat = "CREATE TABLE IF NOT EXISTS " + baseTable + "  ("

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
index 4a21864..77af84c 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ArrayIT.java
@@ -2777,4 +2777,32 @@ public class ArrayIT extends BaseClientManagedTimeIT {
         assertTrue(rs.next());
         assertEquals(conn.createArrayOf("CHAR", new String[]{"aaa", "bbb", "ccc"}), rs.getArray(1));
     }
+
+    @Test
+    public void testArrayIndexFunctionForImmutableTable() throws Exception {
+        String tableName = "testArrayIndexFunctionForImmutableTable".toUpperCase();
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String ddl = "CREATE IMMUTABLE TABLE " + tableName + " (region_name VARCHAR PRIMARY KEY, ZIP VARCHAR ARRAY[10])";
+            conn.createStatement().execute(ddl);
+        }
+        props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().executeUpdate("UPSERT INTO " + tableName + " (region_name,zip) VALUES('SF Bay Area',ARRAY['94115','94030','94125'])");
+            conn.commit();
+        }
+        props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            String sql = "SELECT ZIP[2] FROM " + tableName;
+            try (ResultSet rs = conn.createStatement().executeQuery(sql)) {
+                assertTrue(rs.next());
+                assertEquals("94030", rs.getString(1));
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseJoinIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseJoinIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseJoinIT.java
index f1c1808..152bdf0 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseJoinIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseJoinIT.java
@@ -60,14 +60,14 @@ public abstract class BaseJoinIT extends ParallelStatsDisabledIT {
                 "    \"item_id\" varchar(10), " +
                 "    price integer, " +
                 "    quantity integer, " +
-                "    date timestamp)");
+                "    date timestamp) IMMUTABLE_ROWS=true");
         builder.put(JOIN_CUSTOMER_TABLE_FULL_NAME, "create table " + JOIN_CUSTOMER_TABLE_FULL_NAME +
                 "   (\"customer_id\" varchar(10) not null primary key, " +
                 "    name varchar, " +
                 "    phone varchar(12), " +
                 "    address varchar, " +
                 "    loc_id varchar(5), " +
-                "    date date)");
+                "    date date) IMMUTABLE_ROWS=true");
         builder.put(JOIN_ITEM_TABLE_FULL_NAME, "create table " + JOIN_ITEM_TABLE_FULL_NAME +
                 "   (\"item_id\" varchar(10) not null primary key, " +
                 "    name varchar, " +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
index 9ad12e5..e82daf9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BaseQueryIT.java
@@ -18,7 +18,6 @@
 package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 
 import java.io.IOException;
 import java.sql.Connection;
@@ -30,14 +29,18 @@ import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.atomic.AtomicInteger;
+
 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
+import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.runner.RunWith;
@@ -45,6 +48,7 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 
 
@@ -61,6 +65,21 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
     protected static final String tenantId = getOrganizationId();
     protected static final String ATABLE_INDEX_NAME = "ATABLE_IDX";
     protected static final long BATCH_SIZE = 3;
+    protected static final String[] INDEX_DDLS = new String[] {
+            "CREATE INDEX %s ON %s (a_integer DESC) INCLUDE ("
+                    + "    A_STRING, " + "    B_STRING, " + "    A_DATE)"};
+//    ,
+//            "CREATE INDEX %s ON %s (a_integer, a_string) INCLUDE ("
+//                    + "    B_STRING, " + "    A_DATE)",
+//            "CREATE INDEX %s ON %s (a_integer) INCLUDE ("
+//                    + "    A_STRING, " + "    B_STRING, " + "    A_DATE)",
+//            "CREATE LOCAL INDEX %s ON %s (a_integer DESC) INCLUDE ("
+//                    + "    A_STRING, " + "    B_STRING, " + "    A_DATE)",
+//            "CREATE LOCAL INDEX %s ON %s (a_integer, a_string) INCLUDE (" + "    B_STRING, "
+//                    + "    A_DATE)",
+//            "CREATE LOCAL INDEX %s ON %s (a_integer) INCLUDE ("
+//                    + "    A_STRING, " + "    B_STRING, " + "    A_DATE)", 
+//            "" };
 
     @BeforeClass
     @Shadower(classBeingShadowed = BaseClientManagedTimeIT.class)
@@ -79,42 +98,69 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
     }
     
+    private static Map<Pair<String, String>, Pair<String, String>> namesByParams = Maps.newHashMapWithExpectedSize(10);
+    
     protected long ts;
     protected Date date;
     private String indexDDL;
+    private String tableDDLOptions;
+    protected String tableName;
+    protected String indexName;
     
-    public BaseQueryIT(String indexDDL) {
-        this.indexDDL = indexDDL;
+    public BaseQueryIT(String idxDdl, boolean mutable, boolean columnEncoded) {
+        StringBuilder optionBuilder = new StringBuilder();
+        if (!columnEncoded) {
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
+        if (!mutable) {
+            if (optionBuilder.length()>0)
+                optionBuilder.append(",");
+            optionBuilder.append("IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+        try {
+            this.ts = nextTimestamp();
+            Pair<String, String> runParam = new Pair<>(idxDdl, tableDDLOptions);
+            Pair<String, String> tableIndexNames = namesByParams.get(runParam);
+            if (tableIndexNames == null) {
+                this.tableName = initATableValues(null, tenantId, getDefaultSplits(tenantId), date=new Date(System.currentTimeMillis()), ts, getUrl(), tableDDLOptions);
+                this.indexName = generateUniqueName();
+                namesByParams.put(runParam, new Pair<>(tableName, indexName));
+                if (idxDdl.length() > 0) {
+                    this.indexDDL = String.format(idxDdl, indexName, tableName);
+                    Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+                    props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+                    Connection conn = DriverManager.getConnection(getUrl(), props);
+                    conn.createStatement().execute(this.indexDDL);
+                }
+            } else {
+                this.tableName = tableIndexNames.getFirst();
+                this.indexName = tableIndexNames.getSecond();
+                initATableValues(this.tableName, tenantId, getDefaultSplits(tenantId), date=new Date(System.currentTimeMillis()), ts, getUrl(), tableDDLOptions);
+            }
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
     }
     
     @Before
-    public void initTable() throws Exception {
-         ts = nextTimestamp();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), date=new Date(System.currentTimeMillis()), ts, getUrl());
-        if (indexDDL != null && indexDDL.length() > 0) {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
-            Connection conn = DriverManager.getConnection(getUrl(), props);
-            conn.createStatement().execute(indexDDL);
-        }
+    public void init() throws Exception {
+        this.ts = nextTimestamp();
     }
     
-    @Parameters(name="{0}")
+    @Parameters(name="indexDDL={0},mutable={1},columnEncoded={2}")
     public static Collection<Object> data() {
         List<Object> testCases = Lists.newArrayList();
-        testCases.add(new String[] { "CREATE INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer DESC) INCLUDE ("
-                + "    A_STRING, " + "    B_STRING, " + "    A_DATE)" });
-        testCases.add(new String[] { "CREATE INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer, a_string) INCLUDE ("
-                + "    B_STRING, " + "    A_DATE)" });
-        testCases.add(new String[] { "CREATE INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer) INCLUDE ("
-                + "    A_STRING, " + "    B_STRING, " + "    A_DATE)" });
-        testCases.add(new String[] { "CREATE LOCAL INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer DESC) INCLUDE ("
-                + "    A_STRING, " + "    B_STRING, " + "    A_DATE)" });
-        testCases.add(new String[] { "CREATE LOCAL INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer, a_string) INCLUDE ("
-                + "    B_STRING, " + "    A_DATE)" });
-        testCases.add(new String[] { "CREATE LOCAL INDEX " + ATABLE_INDEX_NAME + " ON aTable (a_integer) INCLUDE ("
-                + "    A_STRING, " + "    B_STRING, " + "    A_DATE)" });
-        testCases.add(new String[] { "" });
+        for (String indexDDL : INDEX_DDLS) {
+            for (boolean mutable : new boolean[]{false}) {
+                for (boolean columnEncoded : new boolean[]{false}) {
+                    testCases.add(new Object[] { indexDDL, mutable, columnEncoded });
+                }
+            }
+        }
         return testCases;
     }
     
@@ -132,4 +178,5 @@ public abstract class BaseQueryIT extends BaseClientManagedTimeIT {
     protected static int nextRunCount() {
         return runCount.getAndAdd(1);
     }
+
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
index 59b0f41..d2b14a9 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CaseStatementIT.java
@@ -44,6 +44,7 @@ import java.util.Properties;
 
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -54,8 +55,8 @@ import com.google.common.collect.Lists;
 @RunWith(Parameterized.class)
 public class CaseStatementIT extends BaseQueryIT {
 
-    public CaseStatementIT(String indexDDL) {
-        super(indexDDL);
+    public CaseStatementIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Parameters(name="CaseStatementIT_{index}") // name is used by failsafe as file name in reports
@@ -65,7 +66,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testSimpleCaseStatement() throws Exception {
-        String query = "SELECT CASE a_integer WHEN 1 THEN 'a' WHEN 2 THEN 'b' WHEN 3 THEN 'c' ELSE 'd' END, entity_id AS a FROM ATABLE WHERE organization_id=? AND a_integer < 6";
+        String query = "SELECT CASE a_integer WHEN 1 THEN 'a' WHEN 2 THEN 'b' WHEN 3 THEN 'c' ELSE 'd' END, entity_id AS a FROM " + tableName + " WHERE organization_id=? AND a_integer < 6";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -88,7 +89,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testMultiCondCaseStatement() throws Exception {
-        String query = "SELECT CASE WHEN a_integer <= 2 THEN 1.5 WHEN a_integer = 3 THEN 2 WHEN a_integer <= 6 THEN 4.5 ELSE 5 END AS a FROM ATABLE WHERE organization_id=?";
+        String query = "SELECT CASE WHEN a_integer <= 2 THEN 1.5 WHEN a_integer = 3 THEN 2 WHEN a_integer <= 6 THEN 4.5 ELSE 5 END AS a FROM " + tableName + " WHERE organization_id=?";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -122,7 +123,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testPartialEvalCaseStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? and CASE WHEN 1234 = a_integer THEN 1 WHEN x_integer = 5 THEN 2 ELSE 3 END = 2";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and CASE WHEN 1234 = a_integer THEN 1 WHEN x_integer = 5 THEN 2 ELSE 3 END = 2";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -140,7 +141,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testFoundIndexOnPartialEvalCaseStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? and CASE WHEN a_integer = 1234 THEN 1 WHEN x_integer = 3 THEN y_integer ELSE 3 END = 300";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and CASE WHEN a_integer = 1234 THEN 1 WHEN x_integer = 3 THEN y_integer ELSE 3 END = 300";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -159,7 +160,7 @@ public class CaseStatementIT extends BaseQueryIT {
     // TODO: we need some tests that have multiple versions of key values
     @Test
     public void testUnfoundMultiColumnCaseStatement() throws Exception {
-        String query = "SELECT entity_id, b_string FROM ATABLE WHERE organization_id=? and CASE WHEN a_integer = 1234 THEN 1 WHEN a_date < ? THEN y_integer WHEN x_integer = 4 THEN 4 ELSE 3 END = 4";
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE organization_id=? and CASE WHEN a_integer = 1234 THEN 1 WHEN a_date < ? THEN y_integer WHEN x_integer = 4 THEN 4 ELSE 3 END = 4";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -177,8 +178,9 @@ public class CaseStatementIT extends BaseQueryIT {
     }
     
     @Test
+    @Ignore // FIXME:PHOENIX-3695
     public void testUnfoundSingleColumnCaseStatement() throws Exception {
-        String query = "SELECT entity_id, b_string FROM ATABLE WHERE organization_id=? and CASE WHEN a_integer = 0 or a_integer != 0 THEN 1 ELSE 0 END = 0";
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE organization_id=? and CASE WHEN a_integer = 0 or a_integer != 0 THEN 1 ELSE 0 END = 0";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -188,8 +190,8 @@ public class CaseStatementIT extends BaseQueryIT {
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
         Connection upsertConn = DriverManager.getConnection(url, props);
         String upsertStmt =
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            "(" +
             "    ENTITY_ID, " +
             "    ORGANIZATION_ID, " +
             "    A_INTEGER) " +
@@ -211,7 +213,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testNonNullMultiCondCaseStatement() throws Exception {
-        String query = "SELECT CASE WHEN entity_id = '000000000000000' THEN 1 WHEN entity_id = '000000000000001' THEN 2 ELSE 3 END FROM ATABLE WHERE organization_id=?";
+        String query = "SELECT CASE WHEN entity_id = '000000000000000' THEN 1 WHEN entity_id = '000000000000001' THEN 2 ELSE 3 END FROM " + tableName + " WHERE organization_id=?";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -228,7 +230,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testNullMultiCondCaseStatement() throws Exception {
-        String query = "SELECT CASE WHEN entity_id = '000000000000000' THEN 1 WHEN entity_id = '000000000000001' THEN 2 END FROM ATABLE WHERE organization_id=?";
+        String query = "SELECT CASE WHEN entity_id = '000000000000000' THEN 1 WHEN entity_id = '000000000000001' THEN 2 END FROM " + tableName + " WHERE organization_id=?";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -245,7 +247,7 @@ public class CaseStatementIT extends BaseQueryIT {
     
     @Test
     public void testNullabilityMultiCondCaseStatement() throws Exception {
-        String query = "SELECT CASE WHEN a_integer <= 2 THEN ? WHEN a_integer = 3 THEN ? WHEN a_integer <= ? THEN ? ELSE 5 END AS a FROM ATABLE WHERE organization_id=?";
+        String query = "SELECT CASE WHEN a_integer <= 2 THEN ? WHEN a_integer = 3 THEN ? WHEN a_integer <= ? THEN ? ELSE 5 END AS a FROM " + tableName + " WHERE organization_id=?";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
index 876c23a..bab18d5 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CastAndCoerceIT.java
@@ -43,8 +43,8 @@ import org.junit.runners.Parameterized.Parameters;
 @RunWith(Parameterized.class)
 public class CastAndCoerceIT extends BaseQueryIT {
 
-    public CastAndCoerceIT(String indexDDL) {
-        super(indexDDL);
+    public CastAndCoerceIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Parameters(name="CastAndCoerceIT_{index}") // name is used by failsafe as file name in reports
@@ -54,7 +54,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
     
     @Test
     public void testCastOperatorInSelect() throws Exception {
-        String query = "SELECT CAST(a_integer AS decimal)/2 FROM aTable WHERE ?=organization_id and 5=a_integer";
+        String query = "SELECT CAST(a_integer AS decimal)/2 FROM " + tableName + " WHERE ?=organization_id and 5=a_integer";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -72,7 +72,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
     
     @Test
     public void testCastOperatorInWhere() throws Exception {
-        String query = "SELECT a_integer FROM aTable WHERE ?=organization_id and 2.5 = CAST(a_integer AS DECIMAL)/2 ";
+        String query = "SELECT a_integer FROM " + tableName + " WHERE ?=organization_id and 2.5 = CAST(a_integer AS DECIMAL)/2 ";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -90,7 +90,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
     
     @Test
     public void testCoerceIntegerToLong() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_long >= x_integer";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND x_long >= x_integer";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -110,7 +110,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
     
     @Test
     public void testCoerceLongToDecimal1() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_decimal > x_integer";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND x_decimal > x_integer";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -128,7 +128,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
     
     @Test
     public void testCoerceLongToDecimal2() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_integer <= x_decimal";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND x_integer <= x_decimal";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -146,7 +146,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
     
     @Test
     public void testCoerceTinyIntToSmallInt() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_byte >= a_short";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_byte >= a_short";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -176,12 +176,12 @@ public class CastAndCoerceIT extends BaseQueryIT {
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3);
         conn = DriverManager.getConnection(url, props);
         conn.setAutoCommit(true);
-        conn.createStatement().execute("UPSERT INTO ATABLE(organization_id,entity_id,a_time,a_timestamp) SELECT organization_id,entity_id,a_date,a_date FROM ATABLE");
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (organization_id,entity_id,a_time,a_timestamp) SELECT organization_id,entity_id,a_date,a_date FROM " + tableName);
 
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5);
         conn = DriverManager.getConnection(url, props);
         try {
-            query = "SELECT entity_id, CAST(a_date AS BIGINT) FROM ATABLE WHERE organization_id=? AND a_date IS NOT NULL LIMIT 1";
+            query = "SELECT entity_id, CAST(a_date AS BIGINT) FROM " + tableName + " WHERE organization_id=? AND a_date IS NOT NULL LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             rs = statement.executeQuery();
@@ -190,7 +190,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             dateAsLong = rs.getLong(2);
             assertFalse(rs.next());
         
-            query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_date = CAST(? AS DATE) LIMIT 1";
+            query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_date = CAST(? AS DATE) LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             statement.setLong(2, dateAsLong);
@@ -199,7 +199,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             assertEquals(ROW1, rs.getString(1));
             assertFalse(rs.next());
 
-            query = "SELECT entity_id, CAST(a_time AS BIGINT) FROM ATABLE WHERE organization_id=? AND a_time IS NOT NULL LIMIT 1";
+            query = "SELECT entity_id, CAST(a_time AS BIGINT) FROM " + tableName + " WHERE organization_id=? AND a_time IS NOT NULL LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             rs = statement.executeQuery();
@@ -208,7 +208,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             dateAsLong = rs.getLong(2);
             assertFalse(rs.next());
         
-            query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_time = CAST(? AS TIME) LIMIT 1";
+            query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_time = CAST(? AS TIME) LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             statement.setLong(2, dateAsLong);
@@ -217,7 +217,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             assertEquals(ROW1, rs.getString(1));
             assertFalse(rs.next());
 
-            query = "SELECT entity_id, CAST(a_timestamp AS DECIMAL) FROM ATABLE WHERE organization_id=? AND a_timestamp IS NOT NULL LIMIT 1";
+            query = "SELECT entity_id, CAST(a_timestamp AS DECIMAL) FROM " + tableName + " WHERE organization_id=? AND a_timestamp IS NOT NULL LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             rs = statement.executeQuery();
@@ -226,7 +226,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             dateAsDecimal = rs.getBigDecimal(2);
             assertFalse(rs.next());
         
-            query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_timestamp = CAST(? AS TIMESTAMP) LIMIT 1";
+            query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_timestamp = CAST(? AS TIMESTAMP) LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             statement.setBigDecimal(2, dateAsDecimal);
@@ -236,7 +236,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             assertFalse(rs.next());
 
 
-            query = "SELECT entity_id, CAST(a_timestamp AS BIGINT) FROM ATABLE WHERE organization_id=? AND a_timestamp IS NOT NULL LIMIT 1";
+            query = "SELECT entity_id, CAST(a_timestamp AS BIGINT) FROM " + tableName + " WHERE organization_id=? AND a_timestamp IS NOT NULL LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             rs = statement.executeQuery();
@@ -245,7 +245,7 @@ public class CastAndCoerceIT extends BaseQueryIT {
             dateAsLong = rs.getLong(2);
             assertFalse(rs.next());
         
-            query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_timestamp = CAST(? AS TIMESTAMP) LIMIT 1";
+            query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_timestamp = CAST(? AS TIMESTAMP) LIMIT 1";
             statement = conn.prepareStatement(query);
             statement.setString(1, tenantId);
             statement.setLong(2, dateAsLong);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
index ea4f3ab..3330a08 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ClientTimeArithmeticQueryIT.java
@@ -60,8 +60,8 @@ import com.google.common.collect.Lists;
 @RunWith(Parameterized.class)
 public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
 
-    public ClientTimeArithmeticQueryIT(String indexDDL) {
-        super(indexDDL);
+    public ClientTimeArithmeticQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Parameters(name="ClientTimeArithmeticQueryIT_{index}") // name is used by failsafe as file name in reports
@@ -71,7 +71,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testDateAdd() throws Exception {
-        String query = "SELECT entity_id, b_string FROM ATABLE WHERE a_date + CAST(0.5 AS DOUBLE) < ?";
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE a_date + CAST(0.5 AS DOUBLE) < ?";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -92,7 +92,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testDecimalAddExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER + X_DECIMAL > 11";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER + X_DECIMAL > 11";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -111,7 +111,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testDoubleAddExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where a_double + a_float > 0.08";
+        String query = "SELECT entity_id FROM " + tableName + " where a_double + a_float > 0.08";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -130,7 +130,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testUnsignedDoubleAddExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where a_unsigned_double + a_unsigned_float > 0.08";
+        String query = "SELECT entity_id FROM " + tableName + " where a_unsigned_double + a_unsigned_float > 0.08";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -153,11 +153,11 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     @Test
     public void testValidArithmetic() throws Exception {
         String[] queries = new String[] { 
-                "SELECT entity_id,organization_id FROM atable where (A_DATE - A_DATE) * 5 < 0",
-                "SELECT entity_id,organization_id FROM atable where 1 + A_DATE  < A_DATE",
-                "SELECT entity_id,organization_id FROM atable where A_DATE - 1 < A_DATE",
-                "SELECT entity_id,organization_id FROM atable where A_INTEGER - 45 < 0",
-                "SELECT entity_id,organization_id FROM atable where X_DECIMAL / 45 < 0", };
+                "SELECT entity_id,organization_id FROM " + tableName + " where (A_DATE - A_DATE) * 5 < 0",
+                "SELECT entity_id,organization_id FROM " + tableName + " where 1 + A_DATE  < A_DATE",
+                "SELECT entity_id,organization_id FROM " + tableName + " where A_DATE - 1 < A_DATE",
+                "SELECT entity_id,organization_id FROM " + tableName + " where A_INTEGER - 45 < 0",
+                "SELECT entity_id,organization_id FROM " + tableName + " where X_DECIMAL / 45 < 0", };
 
         for (String query : queries) {
             Properties props = new Properties();
@@ -175,7 +175,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testIntSubtractionExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER - 4  <= 0";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER - 4  <= 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -189,7 +189,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testDecimalSubtraction1Expression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER - 3.5  <= 0";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER - 3.5  <= 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -203,7 +203,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testDecimalSubtraction2Expression() throws Exception {// check if decimal part makes a difference
-        String query = "SELECT entity_id FROM aTable where X_DECIMAL - 3.5  > 0";
+        String query = "SELECT entity_id FROM " + tableName + " where X_DECIMAL - 3.5  > 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -219,7 +219,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testLongSubtractionExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where X_LONG - 1  < 0";
+        String query = "SELECT entity_id FROM " + tableName + " where X_LONG - 1  < 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -235,7 +235,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testDoubleSubtractionExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where a_double - CAST(0.0002 AS DOUBLE)  < 0";
+        String query = "SELECT entity_id FROM " + tableName + " where a_double - CAST(0.0002 AS DOUBLE)  < 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -251,7 +251,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testSmallIntSubtractionExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where a_short - 129  = 0";
+        String query = "SELECT entity_id FROM " + tableName + " where a_short - 129  = 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -268,7 +268,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testTernarySubtractionExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where  X_INTEGER - X_LONG - 10  < 0";
+        String query = "SELECT entity_id FROM " + tableName + " where  X_INTEGER - X_LONG - 10  < 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -286,7 +286,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testSelectWithSubtractionExpression() throws Exception {
-        String query = "SELECT entity_id, x_integer - 4 FROM aTable where  x_integer - 4 = 0";
+        String query = "SELECT entity_id, x_integer - 4 FROM " + tableName + " where  x_integer - 4 = 0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -303,7 +303,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testConstantSubtractionExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER = 5 - 1 - 2";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER = 5 - 1 - 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -320,7 +320,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testIntDivideExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER / 3 > 2";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER / 3 > 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -337,7 +337,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testDoubleDivideExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where a_double / CAST(3.0 AS DOUBLE) = 0.0003";
+        String query = "SELECT entity_id FROM " + tableName + " where a_double / CAST(3.0 AS DOUBLE) = 0.0003";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -354,7 +354,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testSmallIntDivideExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where a_short / 135 = 1";
+        String query = "SELECT entity_id FROM " + tableName + " where a_short / 135 = 1";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -371,7 +371,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testIntToDecimalDivideExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER / 3.0 > 2";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER / 3.0 > 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -386,7 +386,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testConstantDivideExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER = 9 / 3 / 3";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER = 9 / 3 / 3";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -402,7 +402,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     }
     @Test
     public void testSelectWithDivideExpression() throws Exception {
-        String query = "SELECT entity_id, a_integer/3 FROM aTable where  a_integer = 9";
+        String query = "SELECT entity_id, a_integer/3 FROM " + tableName + " where  a_integer = 9";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -420,7 +420,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testNegateExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER - 4 = -1";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER - 4 = -1";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -437,7 +437,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testIntMultiplyExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER * 2 = 16";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER * 2 = 16";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -454,7 +454,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testDoubleMultiplyExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_DOUBLE * CAST(2.0 AS DOUBLE) = 0.0002";
+        String query = "SELECT entity_id FROM " + tableName + " where A_DOUBLE * CAST(2.0 AS DOUBLE) = 0.0002";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -471,7 +471,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
 
     @Test
     public void testLongMultiplyExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where X_LONG * 2 * 2 = 20";
+        String query = "SELECT entity_id FROM " + tableName + " where X_LONG * 2 * 2 = 20";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -488,7 +488,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
 
     @Test
     public void testIntToDecimalMultiplyExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER * 1.5 > 9";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER * 1.5 > 9";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -504,7 +504,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
 
     @Test
     public void testDecimalMultiplyExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where X_DECIMAL * A_INTEGER > 29.5";
+        String query = "SELECT entity_id FROM " + tableName + " where X_DECIMAL * A_INTEGER > 29.5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -519,7 +519,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testIntAddExpression() throws Exception {
-        String query = "SELECT entity_id FROM aTable where A_INTEGER + 2 = 4";
+        String query = "SELECT entity_id FROM " + tableName + " where A_INTEGER + 2 = 4";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -536,11 +536,11 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testCoalesceFunction() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE coalesce(X_DECIMAL,0.0) = 0.0";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE coalesce(X_DECIMAL,0.0) = 0.0";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO aTable(organization_id,entity_id,x_decimal) values(?,?,?)");
+        PreparedStatement stmt = conn.prepareStatement("UPSERT INTO  " + tableName + " (organization_id,entity_id,x_decimal) values(?,?,?)");
         stmt.setString(1, getOrganizationId());
         stmt.setString(2, ROW1);
         stmt.setBigDecimal(3, BigDecimal.valueOf(1.0));
@@ -576,7 +576,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
     
     @Test
     public void testDateSubtract() throws Exception {
-        String query = "SELECT entity_id, b_string FROM ATABLE WHERE a_date - CAST(0.5 AS DOUBLE) > ?";
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE a_date - CAST(0.5 AS DOUBLE) > ?";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -602,7 +602,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
         
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);
         Connection conn = DriverManager.getConnection(url, props);
-        PreparedStatement statement = conn.prepareStatement("UPSERT INTO ATABLE(organization_id,entity_id,a_time) VALUES(?,?,?)");
+        PreparedStatement statement = conn.prepareStatement("UPSERT INTO  " + tableName + " (organization_id,entity_id,a_time) VALUES(?,?,?)");
         statement.setString(1, getOrganizationId());
         statement.setString(2, ROW2);
         statement.setDate(3, date);
@@ -625,7 +625,7 @@ public class ClientTimeArithmeticQueryIT extends BaseQueryIT {
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 25);
         conn = DriverManager.getConnection(url, props);
         try {
-            statement = conn.prepareStatement("SELECT entity_id, b_string FROM ATABLE WHERE a_date - a_time > 1");
+            statement = conn.prepareStatement("SELECT entity_id, b_string FROM " + tableName + " WHERE a_date - a_time > 1");
             ResultSet rs = statement.executeQuery();
             @SuppressWarnings("unchecked")
             List<List<Object>> expectedResults = Lists.newArrayList(

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java
new file mode 100644
index 0000000..dd64428
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ColumnEncodedBytesPropIT.java
@@ -0,0 +1,95 @@
+package org.apache.phoenix.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+
+public class ColumnEncodedBytesPropIT extends ParallelStatsDisabledIT {
+	
+	private String generateColsDDL(int numCols) {
+		StringBuilder sb = new StringBuilder();
+		for (int i=0; i<numCols; ++i) {
+			if (i>0) {
+				sb.append(" , ");
+			}
+			sb.append("col_").append(i).append(" VARCHAR ");
+		}
+		return sb.toString();
+	}
+	
+	@Test
+	public void testValidateProperty() throws SQLException {
+		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTableFullName1 = SchemaUtil.getTableName("", generateUniqueName());
+        String dataTableFullName2 = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            String ddl = "CREATE TABLE  " + dataTableFullName1 +
+                    "  (id varchar not null, val varchar " + 
+                    "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=4";
+            stmt.execute(ddl);
+            
+            ddl = "CREATE TABLE  " + dataTableFullName2 +
+                    "  (id varchar not null, val varchar " + 
+                    "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=NONE";
+            stmt.execute(ddl);
+            
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable dataTable1 = phxConn.getTable(new PTableKey(null, dataTableFullName1));
+            assertEquals("Encoding scheme set incorrectly", QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS, dataTable1.getEncodingScheme());
+            
+            PTable dataTable2 = phxConn.getTable(new PTableKey(null, dataTableFullName2));
+            assertEquals("Encoding scheme set incorrectly", QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, dataTable2.getEncodingScheme());
+        } 
+	}
+
+	@Test
+	public void testValidateMaxCols() throws SQLException {
+		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String dataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create table with more cols than is supported by the encoding
+            try {
+                stmt.execute("CREATE TABLE  " + dataTableFullName +
+                        "  (id varchar not null, " + generateColsDDL(QualifierEncodingScheme.ONE_BYTE_QUALIFIERS.getMaxQualifier()-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+2) + 
+                        "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=1");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.MAX_COLUMNS_EXCEEDED.getErrorCode(), e.getErrorCode());
+            }
+            
+            // create table with number of cols equal to that supported by the encoding
+            stmt.execute("CREATE TABLE  " + dataTableFullName +
+                    "  (id varchar not null, " + generateColsDDL(QualifierEncodingScheme.ONE_BYTE_QUALIFIERS.getMaxQualifier()-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1) + 
+                    "  CONSTRAINT pk PRIMARY KEY (id)) COLUMN_ENCODED_BYTES=1");
+            
+            // add one more column
+            try {
+                stmt.execute("ALTER TABLE  " + dataTableFullName + " ADD val_x VARCHAR");
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.MAX_COLUMNS_EXCEEDED.getErrorCode(), e.getErrorCode());
+            }
+        } 
+	}
+	
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
index 1539ad7..db4542e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/CountDistinctCompressionIT.java
@@ -51,7 +51,7 @@ public class CountDistinctCompressionIT extends BaseUniqueNamesOwnClusterIT {
     @Test
     public void testDistinctCountOnColumn() throws Exception {
         String tenantId = getOrganizationId();
-        String tableName = initATableValues(null, tenantId, getDefaultSplits(tenantId), (Date)null, null, getUrl());
+        String tableName = initATableValues(null, tenantId, getDefaultSplits(tenantId), (Date)null, null, getUrl(), null);
 
         String query = "SELECT count(DISTINCT A_STRING) FROM " + tableName;
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..633d93c 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
@@ -19,6 +19,7 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.hadoop.hbase.HColumnDescriptor.DEFAULT_REPLICATION_SCOPE;
 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,21 +27,30 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.Properties;
 
 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.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.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.SchemaNotFoundException;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
+import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -502,4 +512,51 @@ public class CreateTableIT extends BaseClientManagedTimeIT {
             fail();
         }
     }
+    
+    @Test
+    public void testCreateTableIfNotExistsForEncodedColumnNames() throws Exception {
+        long ts = nextTimestamp();
+        Properties props = PropertiesUtil.deepCopy(TestUtil.TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
+        String tableName = "testCreateTableIfNotExistsForEncodedColumnNames".toUpperCase();
+        String createTableDDL = "CREATE TABLE IF NOT EXISTS " + tableName + " (pk INTEGER PRIMARY KEY)";
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(createTableDDL);
+            assertColumnEncodingMetadata(QualifierEncodingScheme.TWO_BYTE_QUALIFIERS, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, tableName, conn);
+        }
+        // Execute the ddl again
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(createTableDDL);
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName);
+            assertFalse(rs.next());
+            assertColumnEncodingMetadata(QualifierEncodingScheme.TWO_BYTE_QUALIFIERS, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, tableName, conn);
+        }
+        // Now execute the ddl with a different COLUMN_ENCODED_BYTES. This shouldn't change the original encoded bytes setting.
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(createTableDDL + " COLUMN_ENCODED_BYTES = 1");
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName);
+            assertFalse(rs.next());
+            assertColumnEncodingMetadata(QualifierEncodingScheme.TWO_BYTE_QUALIFIERS, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, tableName, conn);
+        }
+        // Now execute the ddl where COLUMN_ENCODED_BYTES=0. This shouldn't change the original encoded bytes setting.
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(nextTimestamp()));
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
+            conn.createStatement().execute(createTableDDL + " COLUMN_ENCODED_BYTES = 0");
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + tableName);
+            assertFalse(rs.next());
+            assertColumnEncodingMetadata(QualifierEncodingScheme.TWO_BYTE_QUALIFIERS, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, tableName, conn);
+        }
+
+    }
+
+    private void assertColumnEncodingMetadata(QualifierEncodingScheme expectedEncodingScheme,
+            ImmutableStorageScheme expectedStorageScheme, String tableName,
+            Connection conn) throws Exception {
+        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+        PTable table = phxConn.getTable(new PTableKey(null, tableName));
+        assertEquals(expectedEncodingScheme, table.getEncodingScheme());
+        assertEquals(expectedStorageScheme, table.getImmutableStorageScheme());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
index ad45d5e..fb43538 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DateTimeIT.java
@@ -76,7 +76,7 @@ public class DateTimeIT extends ParallelStatsDisabledIT {
     
     private String initAtable() throws SQLException {
         String tableName = generateUniqueName();
-        ensureTableCreated(getUrl(), tableName, ATABLE_NAME, (byte[][])null);
+        ensureTableCreated(getUrl(), tableName, ATABLE_NAME, (byte[][])null, null);
         PreparedStatement stmt = conn.prepareStatement(
             "upsert into " + tableName +
             "(" +

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
index 62d79bc..7c04d01 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DefaultColumnValueIT.java
@@ -37,6 +37,7 @@ import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.DateUtil;
 import org.junit.Before;
+import org.junit.Ignore;
 import org.junit.Test;
 
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
index 67be132..39fb747 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DerivedTableIT.java
@@ -74,7 +74,7 @@ public class DerivedTableIT extends BaseClientManagedTimeIT {
     @Before
     public void initTable() throws Exception {
          ts = nextTimestamp();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         if (indexDDL != null && indexDDL.length > 0) {
             Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
             props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
index 4484fc1..a7482cb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DistinctCountIT.java
@@ -287,9 +287,9 @@ public class DistinctCountIT extends BaseClientManagedTimeIT {
     
     protected static void initATableValues(String tenantId1, String tenantId2, byte[][] splits, Date date, Long ts) throws Exception {
         if (ts == null) {
-            ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, splits);
+            ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, splits, null);
         } else {
-            ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, splits, ts-2);
+            ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, splits, ts-2, null);
         }
         
         Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
index 25e7230..3f02113 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DynamicColumnIT.java
@@ -213,5 +213,68 @@ public class DynamicColumnIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
+    
+    @Test
+    public void testDynamicColumnOnNewTable() throws Exception {
+        String tableName = generateUniqueName();
+        String ddl = "create table " + tableName + 
+            "   (entry varchar not null," +
+            "    F varchar," +
+            "    A.F1v1 varchar," +
+            "    A.F1v2 varchar," +
+            "    B.F2v1 varchar" +
+            "    CONSTRAINT pk PRIMARY KEY (entry))";
+        String dml = "UPSERT INTO " + tableName + " values (?, ?, ?, ?, ?)";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(ddl);
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                stmt.setString(1, "entry");
+                stmt.setString(2, "a");
+                stmt.setString(3, "b");
+                stmt.setString(4, "c");
+                stmt.setString(5, "d");
+                stmt.executeUpdate();
+                conn.commit();
+            }
+            dml = "UPSERT INTO " + tableName + "(entry, F, A.F1V1, A.F1v2, B.F2V1, DYNCOL1 VARCHAR, DYNCOL2 VARCHAR) VALUES (?, ?, ?, ?, ?, ?, ?)";
+            try (PreparedStatement stmt = conn.prepareStatement(dml)) {
+                stmt.setString(1, "dynentry");
+                stmt.setString(2, "a");
+                stmt.setString(3, "b");
+                stmt.setString(4, "c");
+                stmt.setString(5, "d");
+                stmt.setString(6, "e");
+                stmt.setString(7, "f");
+                stmt.executeUpdate();
+                conn.commit();
+            }
+            
+            // test dynamic column in where clause
+            String query = "SELECT entry, F from " + tableName + " (DYNCOL1 VARCHAR, DYNCOL2 VARCHAR) " + " WHERE DYNCOL1 = ?";
+            try (PreparedStatement stmt = conn.prepareStatement(query)) {
+                stmt.setString(1, "e");
+                ResultSet rs = stmt.executeQuery();
+                assertTrue(rs.next());
+                assertEquals("dynentry", rs.getString(1));
+                assertEquals("a", rs.getString(2));
+                assertFalse(rs.next());
+            }
+            
+            // test dynamic column with projection
+            query = "SELECT entry, dyncol1, dyncol2 from " + tableName + " (DYNCOL1 VARCHAR, DYNCOL2 VARCHAR) ";
+            try (PreparedStatement stmt = conn.prepareStatement(query)) {
+                ResultSet rs = stmt.executeQuery();
+                assertTrue(rs.next());
+                assertEquals("dynentry", rs.getString(1));
+                assertEquals("e", rs.getString(2));
+                assertEquals("f", rs.getString(3));
+                assertTrue(rs.next());
+                assertEquals("entry", rs.getString(1));
+                assertEquals(null, rs.getString(2));
+                assertEquals(null, rs.getString(2));
+                assertFalse(rs.next());
+            }
+        }
+    }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
index f5c1df2..f46b3e4 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ExtendedQueryExecIT.java
@@ -52,7 +52,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
         Date date = new Date(1);
         String tenantId = getOrganizationId();
 
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl(), null);
         
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+1));
@@ -75,7 +75,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
     public void testTypeMismatchToDateFunctionBind() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),null, ts, getUrl(), null);
 
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -103,7 +103,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
         Date date = new Date(1);
         String tenantId = getOrganizationId();
 
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl(), null);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+1));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -148,7 +148,7 @@ public class ExtendedQueryExecIT extends BaseClientManagedTimeIT {
         Date date = new Date(1);
         String tenantId = getOrganizationId();
 
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId),date, ts, getUrl(), null);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+1));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
index 26e13eb..131e146 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/FunkyNamesIT.java
@@ -39,7 +39,7 @@ import org.junit.Test;
 public class FunkyNamesIT extends BaseClientManagedTimeIT {
 
     protected static void initTableValues(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(), FUNKY_NAME, FUNKY_NAME,splits, ts-2);
+        ensureTableCreated(getUrl(), FUNKY_NAME, FUNKY_NAME,splits, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);


[07/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiEncodedCQKeyValueComparisonFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiEncodedCQKeyValueComparisonFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiEncodedCQKeyValueComparisonFilter.java
new file mode 100644
index 0000000..00e662f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/MultiEncodedCQKeyValueComparisonFilter.java
@@ -0,0 +1,369 @@
+/*
+ * 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.filter;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.NoSuchElementException;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+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.hbase.util.Writables;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisitor;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.tuple.BaseTuple;
+
+/**
+ * Filter used for tables that use number based column qualifiers generated by one of the encoding schemes in
+ * {@link QualifierEncodingScheme}. Because the qualifiers are number based, instead of using a map of cells to track
+ * the columns that have been found, we can use an array of cells where the index into the array would be derived by the
+ * number based column qualifier. See {@link EncodedCQIncrementalResultTuple}. Using this filter helps us to directly
+ * seek to the next row when the column qualifier that we have encountered is greater than the maxQualifier that we
+ * expect. This helps in speeding up the queries filtering on key value columns.
+ */
+public class MultiEncodedCQKeyValueComparisonFilter extends BooleanExpressionFilter {
+    // Smallest qualifier for the columns that are being projected and filtered on
+    private int minQualifier;
+
+    // Largest qualifier for the columns that are being projected and filtered on
+    private int maxQualifier;
+
+    private QualifierEncodingScheme encodingScheme;
+
+    // Smallest qualifier for the columns in where expression
+    private int whereExpressionMinQualifier;
+
+    // Largest qualifier for the columns in where expression
+    private int whereExpressionMaxQualifier;
+
+    private FilteredKeyValueHolder filteredKeyValues;
+    
+    // BitSet to track the qualifiers in where expression that we expect to find while filtering a row
+    private BitSet whereExpressionQualifiers;
+
+    // Set to track the column families of the columns in where expression
+    private TreeSet<byte[]> cfSet;
+
+    // Boolean that tells us whether the result of expression evaluation as and when we filter key values in a row
+    private Boolean matchedColumn;
+
+    // Tuple used to store the relevant key values found while filtering a row
+    private EncodedCQIncrementalResultTuple inputTuple = new EncodedCQIncrementalResultTuple();
+    
+    // Member variable to cache the size of whereExpressionQualifiers
+    private int expectedCardinality;
+    
+    private static final byte[] UNITIALIZED_KEY_BUFFER = new byte[0];
+    
+    public MultiEncodedCQKeyValueComparisonFilter() {}
+
+    public MultiEncodedCQKeyValueComparisonFilter(Expression expression, QualifierEncodingScheme scheme) {
+        super(expression);
+        checkArgument(scheme != NON_ENCODED_QUALIFIERS, "Filter can only be used for encoded qualifiers");
+        this.encodingScheme = scheme;
+        initFilter(expression);
+    }
+    
+    private final class FilteredKeyValueHolder {
+        // Cell values corresponding to columns in where expression that were found while filtering a row.
+        private Cell[] filteredCells;
+
+        // BitSet to track whether qualifiers in where expression were found when filtering a row
+        private BitSet filteredQualifiers;
+        
+        // Using an explicit counter instead of relying on the cardinality of the bitset as computing the 
+        // cardinality could be slightly more expensive than just incrementing an integer
+        private int numKeyValues;
+        
+        private FilteredKeyValueHolder(int size) {
+            filteredCells = new Cell[size];
+            filteredQualifiers = new BitSet(size);
+        }
+
+        private void setCell(int qualifier, Cell c) {
+            int index = qualifier - whereExpressionMinQualifier;
+            filteredCells[index] = c;
+            filteredQualifiers.set(index);
+            numKeyValues++;
+        }
+
+        private Cell getCell(int qualifier) {
+            int index = qualifier - whereExpressionMinQualifier;
+            return filteredQualifiers.get(index) ? filteredCells[index] : null;
+        }
+
+        private void clear() {
+            // Note here that we are only clearing out the filteredQualifiers bitset. We are not setting all the
+            // entries in filteredKeyValues to null or allocating a new Cell array as that would be expensive.
+            filteredQualifiers.clear();
+            numKeyValues = 0;
+        }
+        
+        /**
+         * This method really shouldn't be the way for getting hold of cells. It was
+         * just added to keep the tuple.get(index) method happy.
+         */
+        public Cell getCellAtIndex(int index) {
+            int bitIndex;
+            for (bitIndex = filteredQualifiers.nextSetBit(0); bitIndex >= 0 && index >= 0; bitIndex = filteredQualifiers
+                    .nextSetBit(bitIndex + 1)) {
+                index--;
+            }
+            if (bitIndex < 0) { throw new NoSuchElementException(); }
+            return filteredCells[bitIndex];
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder(100);
+            int length = filteredQualifiers.length();
+            for (int i = 0; i < length; i++) {
+                sb.append(filteredCells[i].toString());
+            }
+            return sb.toString();
+        }
+        
+        private boolean allColumnsFound() {
+            return numKeyValues == expectedCardinality;
+        }
+        
+        private int numKeyValues() {
+            return numKeyValues;
+        }
+
+    }
+    
+    private void initFilter(Expression expression) {
+        cfSet = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+        final BitSet expressionQualifiers = new BitSet(20);
+        final Pair<Integer, Integer> range = new Pair<>();
+        ExpressionVisitor<Void> visitor = new StatelessTraverseAllExpressionVisitor<Void>() {
+            @Override
+            public Void visit(KeyValueColumnExpression expression) {
+                int qualifier = encodingScheme.decode(expression.getColumnQualifier());
+                if (range.getFirst() == null) {
+                    range.setFirst(qualifier);
+                    range.setSecond(qualifier);
+                } else if (qualifier < range.getFirst()) {
+                    range.setFirst(qualifier);
+                } else if (qualifier > range.getSecond()) {
+                    range.setSecond(qualifier);
+                }
+                cfSet.add(expression.getColumnFamily());
+                expressionQualifiers.set(qualifier);
+                return null;
+            }
+        };
+        expression.accept(visitor);
+        // Set min and max qualifiers for columns in the where expression
+        whereExpressionMinQualifier = range.getFirst();
+        whereExpressionMaxQualifier = range.getSecond();
+        
+        int size = whereExpressionMaxQualifier - whereExpressionMinQualifier + 1;
+        filteredKeyValues = new FilteredKeyValueHolder(size);
+        
+        // Initialize the bitset and mark the qualifiers for columns in where expression
+        whereExpressionQualifiers = new BitSet(size);
+        for (int i = whereExpressionMinQualifier; i <= whereExpressionMaxQualifier; i++) {
+            if (expressionQualifiers.get(i)) {
+                whereExpressionQualifiers.set(i - whereExpressionMinQualifier);
+            }
+        }
+        expectedCardinality = whereExpressionQualifiers.cardinality();
+    }
+    
+    private boolean isQualifierForColumnInWhereExpression(int qualifier) {
+        return qualifier >= whereExpressionMinQualifier ? whereExpressionQualifiers.get(qualifier - whereExpressionMinQualifier) : false;
+    }
+    
+    @Override
+    public ReturnCode filterKeyValue(Cell cell) {
+        if (Boolean.TRUE.equals(this.matchedColumn)) {
+          // We already found and matched the single column, all keys now pass
+          return ReturnCode.INCLUDE_AND_NEXT_COL;
+        }
+        if (Boolean.FALSE.equals(this.matchedColumn)) {
+          // We found all the columns, but did not match the expression, so skip to next row
+          return ReturnCode.NEXT_ROW;
+        }
+        inputTuple.setKey(cell);
+        int qualifier = encodingScheme.decode(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength());
+        if (isQualifierForColumnInWhereExpression(qualifier)) {
+            filteredKeyValues.setCell(qualifier, cell);
+            // We found a new column, so we can re-evaluate
+            this.matchedColumn = this.evaluate(inputTuple);
+            if (this.matchedColumn == null) {
+                if (inputTuple.isImmutable()) {
+                    this.matchedColumn = Boolean.FALSE;
+                } else {
+                    return ReturnCode.INCLUDE_AND_NEXT_COL;
+                }
+            }
+            return this.matchedColumn ? ReturnCode.INCLUDE_AND_NEXT_COL : ReturnCode.NEXT_ROW;
+        }
+        // The qualifier is not one of the qualifiers in the expression. So decide whether
+        // we would need to include it in our result.
+        if (qualifier < minQualifier) {
+            // Qualifier is smaller than the minimum expected qualifier. Look at the next column.
+            return ReturnCode.NEXT_COL;
+        }
+        // TODO: I don't think we would ever hit this case of encountering a greater than what we expect.
+        // Leaving the code commented out here for future reference.
+        // if (qualifier > maxQualifier) {
+            // Qualifier is larger than the max expected qualifier. We are done looking at columns in this row.
+            // return ReturnCode.NEXT_ROW;
+        // }
+        return ReturnCode.INCLUDE_AND_NEXT_COL;
+    }
+
+    @Override
+    public boolean filterRow() {
+        if (this.matchedColumn == null && !inputTuple.isImmutable() && expression.requiresFinalEvaluation()) {
+            inputTuple.setImmutable();
+            this.matchedColumn = this.evaluate(inputTuple);
+        }
+        return ! (Boolean.TRUE.equals(this.matchedColumn));
+    }
+
+    final class EncodedCQIncrementalResultTuple extends BaseTuple {
+        private final ImmutableBytesWritable keyPtr = new ImmutableBytesWritable(UNITIALIZED_KEY_BUFFER);
+        private boolean isImmutable;
+        
+        @Override
+        public boolean isImmutable() {
+            return isImmutable || filteredKeyValues.allColumnsFound();
+        }
+        
+        public void setImmutable() {
+            this.isImmutable = true;
+        }
+        
+        private void setKey(Cell value) {
+            keyPtr.set(value.getRowArray(), value.getRowOffset(), value.getRowLength());
+        }
+        
+        @Override
+        public void getKey(ImmutableBytesWritable ptr) {
+            ptr.set(keyPtr.get(),keyPtr.getOffset(),keyPtr.getLength());
+        }
+        
+        @Override
+        public Cell getValue(byte[] cf, byte[] cq) {
+            int qualifier = encodingScheme.decode(cq);
+            return filteredKeyValues.getCell(qualifier);
+        }
+        
+        @Override
+        public String toString() {
+            return filteredKeyValues.toString();
+        }
+
+        @Override
+        public int size() {
+            return filteredKeyValues.numKeyValues();
+        }
+
+        /**
+         * This method doesn't perform well and shouldn't be the way of
+         * getting hold of elements in the tuple.
+         */
+        @Override
+        public Cell getValue(int index) {
+            return filteredKeyValues.getCellAtIndex(index);
+        }
+
+        @Override
+        public boolean getValue(byte[] family, byte[] qualifier,
+                ImmutableBytesWritable ptr) {
+            Cell cell = getValue(family, qualifier);
+            if (cell == null)
+                return false;
+            ptr.set(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+            return true;
+        }
+        
+        void reset() {
+            isImmutable = false;
+            keyPtr.set(UNITIALIZED_KEY_BUFFER);
+        }
+    }
+    
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.minQualifier = WritableUtils.readVInt(input);
+        this.maxQualifier = WritableUtils.readVInt(input);
+        this.whereExpressionMinQualifier = WritableUtils.readVInt(input);
+        this.whereExpressionMaxQualifier = WritableUtils.readVInt(input);
+        this.encodingScheme = QualifierEncodingScheme.values()[WritableUtils.readVInt(input)];
+        super.readFields(input);
+        initFilter(expression);
+    }
+    
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeVInt(output, minQualifier);
+        WritableUtils.writeVInt(output, maxQualifier);
+        WritableUtils.writeVInt(output, whereExpressionMinQualifier);
+        WritableUtils.writeVInt(output, whereExpressionMaxQualifier);
+        WritableUtils.writeVInt(output, encodingScheme.ordinal());
+        super.write(output);
+    }
+    
+    public void setMinMaxQualifierRange(Pair<Integer, Integer> minMaxQualifiers) {
+        this.minQualifier = minMaxQualifiers.getFirst();
+        this.maxQualifier = minMaxQualifiers.getSecond();
+    }
+    
+    public static MultiEncodedCQKeyValueComparisonFilter parseFrom(final byte [] pbBytes) throws DeserializationException {
+        try {
+            return (MultiEncodedCQKeyValueComparisonFilter)Writables.getWritable(pbBytes, new MultiEncodedCQKeyValueComparisonFilter());
+        } catch (IOException e) {
+            throw new DeserializationException(e);
+        }
+    }
+    
+    @Override
+    public void reset() {
+        filteredKeyValues.clear();
+        matchedColumn = null;
+        inputTuple.reset();
+        super.reset();
+    }
+
+    @Override
+    public boolean isFamilyEssential(byte[] name) {
+        // Only the column families involved in the expression are essential.
+        // The others are for columns projected in the select expression.
+        return cfSet.contains(name);
+    }
+    
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..00ecd9f 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
@@ -94,7 +94,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 +184,7 @@ public abstract class MultiKeyValueComparisonFilter extends BooleanExpressionFil
         ExpressionVisitor<Void> visitor = new StatelessTraverseAllExpressionVisitor<Void>() {
             @Override
             public Void visit(KeyValueColumnExpression expression) {
-                inputTuple.addColumn(expression.getColumnFamily(), expression.getColumnName());
+                inputTuple.addColumn(expression.getColumnFamily(), expression.getColumnQualifier());
                 return null;
             }
         };
@@ -231,7 +231,7 @@ public abstract class MultiKeyValueComparisonFilter extends BooleanExpressionFil
         return ! (Boolean.TRUE.equals(this.matchedColumn));
     }
 
-      @Override
+    @Override
     public void reset() {
         matchedColumn = null;
         inputTuple.reset();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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..ae3557d 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.SingleCellColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.visitor.StatelessTraverseAllExpressionVisitor;
 import org.apache.phoenix.expression.visitor.TraverseAllExpressionVisitor;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
+import org.apache.phoenix.schema.tuple.Tuple;
 
 
 
@@ -58,7 +60,7 @@ public abstract class SingleKeyValueComparisonFilter extends BooleanExpressionFi
             @Override
             public Void visit(KeyValueColumnExpression expression) {
                 cf = expression.getColumnFamily();
-                cq = expression.getColumnName();
+                cq = expression.getColumnQualifier();
                 return null;
             }
         };

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
index 237ed75..6061dd9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMaintainer.java
@@ -17,7 +17,10 @@
  */
 package org.apache.phoenix.index;
 
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.DataInput;
 import java.io.DataInputStream;
 import java.io.DataOutput;
@@ -29,9 +32,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 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;
@@ -43,17 +48,24 @@ import org.apache.hadoop.hbase.client.Delete;
 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.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.compile.ColumnResolver;
 import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.IndexExpressionCompiler;
 import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.coprocessor.generated.ServerCachingProtos;
+import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo;
 import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.ExpressionType;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.expression.visitor.KeyValueExpressionVisitor;
 import org.apache.phoenix.hbase.index.ValueGetter;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
@@ -67,14 +79,17 @@ 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.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SaltingUtil;
@@ -82,10 +97,12 @@ 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.util.BitSet;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
@@ -93,6 +110,7 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 import org.apache.tephra.TxConstants;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
@@ -105,10 +123,10 @@ import com.google.common.collect.Sets;
  * row and caches any covered columns. Client-side serializes into byte array using 
  * @link #serialize(PTable, ImmutableBytesWritable)}
  * and transmits to server-side through either the 
- * {@link org.apache.phoenix.index.PhoenixIndexCodec#INDEX_MD}
+ * {@link org.apache.phoenix.index.PhoenixIndexCodec#INDEX_PROTO_MD}
  * Mutation attribute or as a separate RPC call using 
  * {@link org.apache.phoenix.cache.ServerCacheClient})
- *
+ * 
  * 
  * @since 2.1.0
  */
@@ -116,8 +134,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
 
     private static final int EXPRESSION_NOT_PRESENT = -1;
     private static final int ESTIMATED_EXPRESSION_SIZE = 8;
-
-	public static IndexMaintainer create(PTable dataTable, PTable index, PhoenixConnection connection) {
+    
+    public static IndexMaintainer create(PTable dataTable, PTable index, PhoenixConnection connection) {
         if (dataTable.getType() == PTableType.INDEX || index.getType() != PTableType.INDEX || !dataTable.getIndexes().contains(index)) {
             throw new IllegalArgumentException();
         }
@@ -179,14 +197,12 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             }
         }
         int nIndexes = 0;
-        int estimatedSize = dataTable.getRowKeySchema().getEstimatedByteSize() + 2;
         while (indexesItr.hasNext()) {
             nIndexes++;
-            PTable index = indexesItr.next();
-            estimatedSize += index.getIndexMaintainer(dataTable, connection).getEstimatedByteSize();
+            indexesItr.next();
         }
-        TrustedByteArrayOutputStream stream = new TrustedByteArrayOutputStream(estimatedSize + 1);
-        DataOutput output = new DataOutputStream(stream);
+        ByteArrayOutputStream stream = new ByteArrayOutputStream();
+        DataOutputStream output = new DataOutputStream(stream);
         try {
             // Encode data table salting in sign of number of indexes
             WritableUtils.writeVInt(output, nIndexes * (dataTable.getBucketNum() == null ? 1 : -1));
@@ -196,15 +212,23 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
                     dataTable.isImmutableRows() ? enabledLocalIndexIterator(indexes.iterator())
                             : nonDisabledIndexIterator(indexes.iterator());
             while (indexesItr.hasNext()) {
-                    indexesItr.next().getIndexMaintainer(dataTable, connection).write(output);
+                    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.IndexMaintainer proto = IndexMaintainer.toProto(indexesItr.next().getIndexMaintainer(dataTable, connection));
+                    byte[] protoBytes = proto.toByteArray();
+                    WritableUtils.writeVInt(output, protoBytes.length);
+                    output.write(protoBytes);
             }
         } catch (IOException e) {
             throw new RuntimeException(e); // Impossible
         }
-        ptr.set(stream.getBuffer(), 0, stream.size());
+        ptr.set(stream.toByteArray(), 0, stream.size());
     }
     
-
+    /**
+     * For client-side to append serialized IndexMaintainers of keyValueIndexes
+     * @param dataTable data table
+     * @param indexMetaDataPtr bytes pointer to hold returned serialized value
+     * @param keyValueIndexes indexes to serialize
+     */
     public static void serializeAdditional(PTable table, ImmutableBytesWritable indexMetaDataPtr,
             List<PTable> keyValueIndexes, PhoenixConnection connection) {
         int nMutableIndexes = indexMetaDataPtr.getLength() == 0 ? 0 : ByteUtil.vintFromBytes(indexMetaDataPtr);
@@ -230,7 +254,10 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             }
             // Serialize mutable indexes afterwards
             for (PTable index : keyValueIndexes) {
-                index.getIndexMaintainer(table, connection).write(output);
+                IndexMaintainer maintainer = index.getIndexMaintainer(table, connection);
+                byte[] protoBytes = IndexMaintainer.toProto(maintainer).toByteArray();
+                WritableUtils.writeVInt(output, protoBytes.length);
+                output.write(protoBytes);
             }
         } catch (IOException e) {
             throw new RuntimeException(e); // Impossible
@@ -239,15 +266,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     }
     
     public static List<IndexMaintainer> deserialize(ImmutableBytesWritable metaDataPtr,
-            KeyValueBuilder builder) {
-        return deserialize(metaDataPtr.get(), metaDataPtr.getOffset(), metaDataPtr.getLength());
+            KeyValueBuilder builder, boolean useProtoForIndexMaintainer) {
+        return deserialize(metaDataPtr.get(), metaDataPtr.getOffset(), metaDataPtr.getLength(), useProtoForIndexMaintainer);
     }
     
-    public static List<IndexMaintainer> deserialize(byte[] buf) {
-        return deserialize(buf, 0, buf.length);
+    public static List<IndexMaintainer> deserialize(byte[] buf, boolean useProtoForIndexMaintainer) {
+        return deserialize(buf, 0, buf.length, useProtoForIndexMaintainer);
     }
 
-    public static List<IndexMaintainer> deserialize(byte[] buf, int offset, int length) {
+    private static List<IndexMaintainer> deserialize(byte[] buf, int offset, int length, boolean useProtoForIndexMaintainer) {
         ByteArrayInputStream stream = new ByteArrayInputStream(buf, offset, length);
         DataInput input = new DataInputStream(stream);
         List<IndexMaintainer> maintainers = Collections.emptyList();
@@ -259,25 +286,31 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             rowKeySchema.readFields(input);
             maintainers = Lists.newArrayListWithExpectedSize(size);
             for (int i = 0; i < size; i++) {
-                IndexMaintainer maintainer = new IndexMaintainer(rowKeySchema, isDataTableSalted);
-                maintainer.readFields(input);
-                maintainers.add(maintainer);
+                if (useProtoForIndexMaintainer) {
+                  int protoSize = WritableUtils.readVInt(input);
+                  byte[] b = new byte[protoSize];
+                  input.readFully(b);
+                  org.apache.phoenix.coprocessor.generated.ServerCachingProtos.IndexMaintainer proto = ServerCachingProtos.IndexMaintainer.parseFrom(b);
+                  maintainers.add(IndexMaintainer.fromProto(proto, rowKeySchema, isDataTableSalted));
+                } else {
+                    IndexMaintainer maintainer = new IndexMaintainer(rowKeySchema, isDataTableSalted);
+                    maintainer.readFields(input);
+                    maintainers.add(maintainer);
+                }
             }
         } catch (IOException e) {
             throw new RuntimeException(e); // Impossible
         }
         return maintainers;
     }
-
+    
     private byte[] viewIndexId;
     private boolean isMultiTenant;
     // indexed expressions that are not present in the row key of the data table, the expression can also refer to a regular column
     private List<Expression> indexedExpressions;
     // 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;
+    
     // 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,12 +324,10 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     private boolean indexWALDisabled;
     private boolean isLocalIndex;
     private boolean immutableRows;
-
     // Transient state
     private final boolean isDataTableSalted;
     private final RowKeySchema dataRowKeySchema;
     
-    private List<ImmutableBytesPtr> indexQualifiers;
     private int estimatedIndexRowKeyBytes;
     private int estimatedExpressionSize;
     private int[] dataPkPosition;
@@ -304,26 +335,48 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     private ColumnReference dataEmptyKeyValueRef;
     private boolean rowKeyOrderOptimizable;
     
+    /**** START: New member variables added in 4.10 *****/ 
+    private QualifierEncodingScheme encodingScheme;
+    private ImmutableStorageScheme immutableStorageScheme;
+    /*
+     * Information for columns of data tables that are being indexed. The first part of the pair is column family name
+     * and second part is the column name. The reason we need to track this state is because for certain storage schemes
+     * like ImmutableStorageScheme#SINGLE_CELL_ARRAY_WITH_OFFSETS, the column for which we need to generate an index
+     * table put/delete is different from the columns that are indexed in the phoenix schema. This information helps us
+     * determine whether or not certain operations like DROP COLUMN should impact the index.
+     */
+    private Set<Pair<String, String>> indexedColumnsInfo;
+    /*
+     * 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.
+     */
+    private Map<ColumnReference, ColumnReference> coveredColumnsMap;
+    /**** END: New member variables added in 4.10 *****/
+
     private IndexMaintainer(RowKeySchema dataRowKeySchema, boolean isDataTableSalted) {
         this.dataRowKeySchema = dataRowKeySchema;
         this.isDataTableSalted = isDataTableSalted;
     }
-
-    private IndexMaintainer(PTable dataTable, PTable index, PhoenixConnection connection) {
+    
+    private IndexMaintainer(final PTable dataTable, final PTable index, PhoenixConnection connection) {
         this(dataTable.getRowKeySchema(), dataTable.getBucketNum() != null);
         assert(dataTable.getType() == PTableType.SYSTEM || dataTable.getType() == PTableType.TABLE || dataTable.getType() == PTableType.VIEW);
         this.rowKeyOrderOptimizable = index.rowKeyOrderOptimizable();
         this.isMultiTenant = dataTable.isMultiTenant();
         this.viewIndexId = index.getViewIndexId() == null ? null : MetaDataUtil.getViewIndexIdDataType().toBytes(index.getViewIndexId());
         this.isLocalIndex = index.getIndexType() == IndexType.LOCAL;
-
+        this.encodingScheme = index.getEncodingScheme();
+        
+        // null check for b/w compatibility
+        this.encodingScheme = index.getEncodingScheme() == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : index.getEncodingScheme();
+        this.immutableStorageScheme = index.getImmutableStorageScheme() == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : index.getImmutableStorageScheme();
+        
         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 +387,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.getColumnForColumnName(dataColumnName) : dataTable.getColumnFamily(dataFamilyName).getPColumnForColumnName(dataColumnName);
                 if (SchemaUtil.isPKColumn(dataColumn)) 
                     continue;
             } catch (ColumnNotFoundException e) {
@@ -366,8 +419,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         this.indexTableName = indexTableName;
         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);
@@ -397,6 +449,8 @@ 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);
+        
         IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
         for (int i = indexPosOffset; i < index.getPKColumns().size(); i++) {
             PColumn indexColumn = index.getPKColumns().get(i);
@@ -409,12 +463,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 +479,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 +488,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(SingleCellColumnExpression expression) {
+                        return addDataColInfo(dataTable, expression);
+                    }
+
+                    private Void addDataColInfo(final PTable dataTable, Expression expression) {
+                        Preconditions.checkArgument(expression instanceof SingleCellColumnExpression
+                                || expression instanceof KeyValueColumnExpression);
+
+                        KeyValueColumnExpression colExpression = null;
+                        if (expression instanceof SingleCellColumnExpression) {
+                            colExpression =
+                                    ((SingleCellColumnExpression) expression).getKeyValueExpression();
+                        } else {
+                            colExpression = ((KeyValueColumnExpression) expression);
+                        }
+                        byte[] cf = colExpression.getColumnFamily();
+                        byte[] cq = colExpression.getColumnQualifier();
+                        try {
+                            PColumn dataColumn =
+                                    cf == null ? dataTable.getColumnForColumnQualifier(null, cq)
+                                            : dataTable.getColumnFamily(cf)
+                                                    .getPColumnForColumnQualifier(cq);
+                            indexedColumnsInfo.add(new Pair<>(dataColumn.getFamilyName()
+                                    .getString(), dataColumn.getName().getString()));
+                        } catch (ColumnNotFoundException | ColumnFamilyNotFoundException
+                                | AmbiguousColumnException e) {
+                            throw new RuntimeException(e);
+                        }
+                        return null;
+                    }
+                };
+                expression.accept(kvVisitor);
             }
             // set the sort order of the expression correctly
             if (indexColumn.getSortOrder() == SortOrder.DESC) {
@@ -442,18 +537,17 @@ 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 = dataColumn.getColumnQualifierBytes();
+                byte[] indexColumnCq = indexColumn.getColumnQualifierBytes();
+                this.coveredColumnsMap.put(new ColumnReference(dataColumn.getFamilyName().getBytes(), dataColumnCq), 
+                        new ColumnReference(indexColumn.getFamilyName().getBytes(), indexColumnCq));
             }
         }
         this.estimatedIndexRowKeyBytes = estimateIndexRowKeyByteSize(indexColByteSize);
         initCachedState();
     }
-
+    
     public byte[] buildRowKey(ValueGetter valueGetter, ImmutableBytesWritable rowKeyPtr, byte[] regionStartKey, byte[] regionEndKey)  {
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         boolean prependRegionStartKey = isLocalIndex && regionStartKey != null;
@@ -854,37 +948,106 @@ 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 {
+        byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey);
         Put put = null;
         // New row being inserted: add the empty key value
         if (valueGetter.getLatestValue(dataEmptyKeyValueRef) == null) {
-            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey);
             put = new Put(indexRowKey);
             // add the keyvalue for the empty row
             put.add(kvBuilder.buildPut(new ImmutableBytesPtr(indexRowKey),
-                this.getEmptyKeyValueFamily(), QueryConstants.EMPTY_COLUMN_BYTES_PTR, ts,
+                this.getEmptyKeyValueFamily(), dataEmptyKeyValueRef.getQualifierWritable(), ts,
                 // set the value to the empty column name
-                QueryConstants.EMPTY_COLUMN_BYTES_PTR));
+                dataEmptyKeyValueRef.getQualifierWritable()));
             put.setDurability(!indexWALDisabled ? Durability.USE_DEFAULT : Durability.SKIP_WAL);
         }
-        int i = 0;
-        for (ColumnReference ref : this.getCoveredColumns()) {
-            ImmutableBytesPtr cq = this.indexQualifiers.get(i++);
-            ImmutableBytesWritable value = valueGetter.getLatestValue(ref);
-            byte[] indexRowKey = this.buildRowKey(valueGetter, dataRowKeyPtr, regionStartKey, regionEndKey);
-            ImmutableBytesPtr rowKey = new ImmutableBytesPtr(indexRowKey);
-            if (value != null) {
+        ImmutableBytesPtr rowKey = new ImmutableBytesPtr(indexRowKey);
+        if (immutableStorageScheme != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
+            // map from index column family to list of pair of index column and data column (for covered columns)
+            Map<ImmutableBytesPtr, List<Pair<ColumnReference, ColumnReference>>> familyToColListMap = Maps.newHashMap();
+            for (ColumnReference ref : this.getCoveredColumns()) {
+                ColumnReference indexColRef = this.coveredColumnsMap.get(ref);
+                ImmutableBytesPtr cf = new ImmutableBytesPtr(indexColRef.getFamily());
+                if (!familyToColListMap.containsKey(cf)) {
+                    familyToColListMap.put(cf, Lists.<Pair<ColumnReference, ColumnReference>>newArrayList());
+                }
+                familyToColListMap.get(cf).add(Pair.newPair(indexColRef, ref));
+            }
+            // iterate over each column family and create a byte[] containing all the columns 
+            for (Entry<ImmutableBytesPtr, List<Pair<ColumnReference, ColumnReference>>> entry : familyToColListMap.entrySet()) {
+                byte[] columnFamily = entry.getKey().copyBytesIfNecessary();
+                List<Pair<ColumnReference, ColumnReference>> colRefPairs = entry.getValue();
+                int maxEncodedColumnQualifier = Integer.MIN_VALUE;
+                // find the max col qualifier
+                for (Pair<ColumnReference, ColumnReference> colRefPair : colRefPairs) {
+                    maxEncodedColumnQualifier = Math.max(maxEncodedColumnQualifier, encodingScheme.decode(colRefPair.getFirst().getQualifier()));
+                }
+                Expression[] colValues = EncodedColumnsUtil.createColumnExpressionArray(maxEncodedColumnQualifier);
+                // set the values of the columns
+                for (Pair<ColumnReference, ColumnReference> colRefPair : colRefPairs) {
+                    ColumnReference indexColRef = colRefPair.getFirst();
+                    ColumnReference dataColRef = colRefPair.getSecond();
+                    Expression expression = new SingleCellColumnExpression(new PDatum() {
+                        @Override
+                        public boolean isNullable() {
+                            return false;
+                        }
+                        
+                        @Override
+                        public SortOrder getSortOrder() {
+                            return null;
+                        }
+                        
+                        @Override
+                        public Integer getScale() {
+                            return null;
+                        }
+                        
+                        @Override
+                        public Integer getMaxLength() {
+                            return null;
+                        }
+                        
+                        @Override
+                        public PDataType getDataType() {
+                            return null;
+                        }
+                    }, dataColRef.getFamily(), dataColRef.getQualifier(), encodingScheme);
+                    ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+                    expression.evaluate(new ValueGetterTuple(valueGetter), ptr);
+                    byte[] value = ptr.copyBytesIfNecessary();
+                    if (value != null) {
+                        int indexArrayPos = encodingScheme.decode(indexColRef.getQualifier())-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
+                        colValues[indexArrayPos] = new LiteralExpression(value);
+                    }
+                }
+                
+                List<Expression> children = Arrays.asList(colValues);
+                // we use SingleCellConstructorExpression to serialize multiple columns into a single byte[]
+                SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                singleCellConstructorExpression.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, QueryConstants.SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES_PTR, ts, ptr));
+            }
+        } else {
+            for (ColumnReference ref : this.getCoveredColumns()) {
+                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));
                 }
             }
         }
@@ -962,7 +1125,6 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         return buildDeleteMutation(kvBuilder, null, dataRowKeyPtr, Collections.<KeyValue>emptyList(), ts, null, null);
     }
     
-    @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);
         // Delete the entire row if any of the indexed columns changed
@@ -972,15 +1134,12 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             Delete delete = new Delete(indexRowKey);
             
             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) {
@@ -992,34 +1151,35 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             return delete;
         }
         Delete delete = null;
+        Set<ColumnReference> dataTableColRefs = coveredColumnsMap.keySet();
         // Delete columns for missing key values
         for (Cell kv : pendingUpdates) {
             if (kv.getTypeByte() != KeyValue.Type.Put.getCode()) {
                 ColumnReference ref = new ColumnReference(kv.getFamily(), kv.getQualifier());
-                if (coveredColumns.contains(ref)) {
+                if (dataTableColRefs.contains(ref)) {
                     if (delete == null) {
                         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()) { 
+                        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);
                     }
                 }
             }
         }
         return delete;
     }
-
+    
     public byte[] getIndexTableName() {
         return indexTableName;
     }
     
     public Set<ColumnReference> getCoveredColumns() {
-        return coveredColumns;
+        return coveredColumnsMap.keySet();
     }
 
     public Set<ColumnReference> getAllColumns() {
@@ -1032,7 +1192,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         // If if there are no covered columns, we know it's our default name
         return emptyKeyValueCFPtr;
     }
-
+    
+    @Deprecated // Only called by code older than our 4.10 release
     @Override
     public void readFields(DataInput input) throws IOException {
         int encodedIndexSaltBucketsAndMultiTenant = WritableUtils.readVInt(input);
@@ -1060,16 +1221,15 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         int encodedCoveredolumnsAndLocalIndex = WritableUtils.readVInt(input);
         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);
+            ColumnReference dataTableRef = new ColumnReference(dataTableCf, dataTableCq);
+            byte[] indexTableCf = isLocalIndex ? IndexUtil.getLocalIndexColumnFamily(dataTableCf) : dataTableCf;
+            byte[] indexTableCq  = IndexUtil.getIndexColumnName(dataTableCf, dataTableCq);
+            ColumnReference indexTableRef = new ColumnReference(indexTableCf, indexTableCq);
+            coveredColumnsMap.put(dataTableRef, indexTableRef);
         }
         // Hack to serialize whether the index row key is optimizable
         int len = WritableUtils.readVInt(input);
@@ -1097,9 +1257,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             int numIndexedExpressions = WritableUtils.readVInt(input);
             indexedExpressions = Lists.newArrayListWithExpectedSize(numIndexedExpressions);        
             for (int i = 0; i < numIndexedExpressions; i++) {
-            	Expression expression = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
-            	expression.readFields(input);
-            	indexedExpressions.add(expression);
+                Expression expression = ExpressionType.values()[WritableUtils.readVInt(input)].newInstance();
+                expression.readFields(input);
+                indexedExpressions.add(expression);
             }
         }
         else {
@@ -1151,6 +1311,79 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         initCachedState();
     }
     
+        
+    public static IndexMaintainer fromProto(ServerCachingProtos.IndexMaintainer proto, RowKeySchema dataTableRowKeySchema, boolean isDataTableSalted) throws IOException {
+        IndexMaintainer maintainer = new IndexMaintainer(dataTableRowKeySchema, isDataTableSalted);
+        maintainer.nIndexSaltBuckets = proto.getSaltBuckets();
+        maintainer.isMultiTenant = proto.getIsMultiTenant();
+        maintainer.viewIndexId = proto.hasViewIndexId() ? proto.getViewIndexId().toByteArray() : null;
+        List<ServerCachingProtos.ColumnReference> indexedColumnsList = proto.getIndexedColumnsList();
+        maintainer.indexedColumns = new HashSet<ColumnReference>(indexedColumnsList.size());
+        for (ServerCachingProtos.ColumnReference colRefFromProto : indexedColumnsList) {
+            maintainer.indexedColumns.add(new ColumnReference(colRefFromProto.getFamily().toByteArray(), colRefFromProto.getQualifier().toByteArray()));
+        }
+        List<Integer> indexedColumnTypes = proto.getIndexedColumnTypeOrdinalList();
+        maintainer.indexedColumnTypes = new ArrayList<PDataType>(indexedColumnTypes.size());
+        for (Integer typeOrdinal : indexedColumnTypes) {
+            maintainer.indexedColumnTypes.add(PDataType.values()[typeOrdinal]);
+        }
+        maintainer.indexTableName = proto.getIndexTableName().toByteArray();
+        maintainer.rowKeyOrderOptimizable = proto.getRowKeyOrderOptimizable();
+        maintainer.dataEmptyKeyValueCF = proto.getDataTableEmptyKeyValueColFamily().toByteArray();
+        ServerCachingProtos.ImmutableBytesWritable emptyKeyValueColFamily = proto.getEmptyKeyValueColFamily();
+        maintainer.emptyKeyValueCFPtr = new ImmutableBytesPtr(emptyKeyValueColFamily.getByteArray().toByteArray(), emptyKeyValueColFamily.getOffset(), emptyKeyValueColFamily.getLength());
+        maintainer.indexedExpressions = new ArrayList<>();
+        try (ByteArrayInputStream stream = new ByteArrayInputStream(proto.getIndexedExpressions().toByteArray())) {
+            DataInput input = new DataInputStream(stream);
+            while (stream.available() > 0) {
+                int expressionOrdinal = WritableUtils.readVInt(input);
+                Expression expression = ExpressionType.values()[expressionOrdinal].newInstance();
+                expression.readFields(input);
+                maintainer.indexedExpressions.add(expression);
+            }
+        }
+        maintainer.rowKeyMetaData = newRowKeyMetaData(maintainer, dataTableRowKeySchema, maintainer.indexedExpressions.size(), isDataTableSalted, maintainer.isMultiTenant);
+        try (ByteArrayInputStream stream = new ByteArrayInputStream(proto.getRowKeyMetadata().toByteArray())) {
+            DataInput input = new DataInputStream(stream);
+            maintainer.rowKeyMetaData.readFields(input);   
+        }
+        maintainer.nDataCFs = proto.getNumDataTableColFamilies();
+        maintainer.indexWALDisabled = proto.getIndexWalDisabled();
+        maintainer.estimatedIndexRowKeyBytes = proto.getIndexRowKeyByteSize();
+        maintainer.immutableRows = proto.getImmutable();
+        List<ColumnInfo> indexedColumnInfoList = proto.getIndexedColumnInfoList();
+        maintainer.indexedColumnsInfo = Sets.newHashSet();
+        for (ColumnInfo info : indexedColumnInfoList) {
+            maintainer.indexedColumnsInfo.add(new Pair<>(info.getFamilyName(), info.getColumnName()));
+        }
+        // proto doesn't support single byte so need an explicit cast here
+        maintainer.encodingScheme = PTable.QualifierEncodingScheme.fromSerializedValue((byte)proto.getEncodingScheme());
+        maintainer.immutableStorageScheme = PTable.ImmutableStorageScheme.fromSerializedValue((byte)proto.getImmutableStorageScheme());
+        maintainer.isLocalIndex = proto.getIsLocalIndex();
+        
+        List<ServerCachingProtos.ColumnReference> dataTableColRefsForCoveredColumnsList = proto.getDataTableColRefForCoveredColumnsList();
+        List<ServerCachingProtos.ColumnReference> indexTableColRefsForCoveredColumnsList = proto.getIndexTableColRefForCoveredColumnsList();
+        maintainer.coveredColumnsMap = Maps.newHashMapWithExpectedSize(dataTableColRefsForCoveredColumnsList.size());
+        boolean encodedColumnNames = maintainer.encodingScheme != NON_ENCODED_QUALIFIERS;
+        Iterator<ServerCachingProtos.ColumnReference> indexTableColRefItr = indexTableColRefsForCoveredColumnsList.iterator();
+        for (ServerCachingProtos.ColumnReference colRefFromProto : dataTableColRefsForCoveredColumnsList) {
+            ColumnReference dataTableColRef = new ColumnReference(colRefFromProto.getFamily().toByteArray(), colRefFromProto.getQualifier( ).toByteArray());
+            ColumnReference indexTableColRef;
+            if (encodedColumnNames) {
+                ServerCachingProtos.ColumnReference fromProto = indexTableColRefItr.next(); 
+                indexTableColRef = new ColumnReference(fromProto.getFamily().toByteArray(), fromProto.getQualifier( ).toByteArray());
+            } else {
+                byte[] cq = IndexUtil.getIndexColumnName(dataTableColRef.getFamily(), dataTableColRef.getQualifier());
+                byte[] cf = maintainer.isLocalIndex ? IndexUtil.getLocalIndexColumnFamily(dataTableColRef.getFamily()) : dataTableColRef.getFamily();
+                indexTableColRef = new ColumnReference(cf, cq);
+            }
+            maintainer.coveredColumnsMap.put(dataTableColRef, indexTableColRef);
+        }
+        maintainer.initCachedState();
+        return maintainer;
+    }
+    
+    @Deprecated // Only called by code older than our 4.10 release
     @Override
     public void write(DataOutput output) throws IOException {
         // Encode nIndexSaltBuckets and isMultiTenant together
@@ -1170,8 +1403,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             WritableUtils.writeVInt(output, type.ordinal());
         }
         // 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) {
+        WritableUtils.writeVInt(output, (coveredColumnsMap.size() + 1) * (isLocalIndex ? -1 : 1));
+        for (ColumnReference ref : coveredColumnsMap.keySet()) {
             Bytes.writeByteArray(output, ref.getFamily());
             Bytes.writeByteArray(output, ref.getQualifier());
         }
@@ -1186,8 +1419,8 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         
         WritableUtils.writeVInt(output, indexedExpressions.size());
         for (Expression expression : indexedExpressions) {
-        	WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
-        	expression.write(output);
+            WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
+            expression.write(output);
         }
         
         rowKeyMetaData.write(output);
@@ -1196,6 +1429,76 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
         // Encode estimatedIndexRowKeyBytes and immutableRows together.
         WritableUtils.writeVInt(output, estimatedIndexRowKeyBytes * (immutableRows ? -1 : 1));
     }
+    
+    public static ServerCachingProtos.IndexMaintainer toProto(IndexMaintainer maintainer) throws IOException {
+        ServerCachingProtos.IndexMaintainer.Builder builder = ServerCachingProtos.IndexMaintainer.newBuilder();
+        builder.setSaltBuckets(maintainer.nIndexSaltBuckets);
+        builder.setIsMultiTenant(maintainer.isMultiTenant);
+        if (maintainer.viewIndexId != null) {
+            builder.setViewIndexId(ByteStringer.wrap(maintainer.viewIndexId));
+        }
+        for (ColumnReference colRef : maintainer.indexedColumns) {
+            ServerCachingProtos.ColumnReference.Builder cRefBuilder =  ServerCachingProtos.ColumnReference.newBuilder();
+            cRefBuilder.setFamily(ByteStringer.wrap(colRef.getFamily()));
+            cRefBuilder.setQualifier(ByteStringer.wrap(colRef.getQualifier()));
+            builder.addIndexedColumns(cRefBuilder.build());
+        }
+        for (PDataType dataType : maintainer.indexedColumnTypes) {
+            builder.addIndexedColumnTypeOrdinal(dataType.ordinal());
+        }
+        for (Entry<ColumnReference, ColumnReference> e : maintainer.coveredColumnsMap.entrySet()) {
+            ServerCachingProtos.ColumnReference.Builder cRefBuilder =  ServerCachingProtos.ColumnReference.newBuilder();
+            ColumnReference dataTableColRef = e.getKey();
+            cRefBuilder.setFamily(ByteStringer.wrap(dataTableColRef.getFamily()));
+            cRefBuilder.setQualifier(ByteStringer.wrap(dataTableColRef.getQualifier()));
+            builder.addDataTableColRefForCoveredColumns(cRefBuilder.build());
+            if (maintainer.encodingScheme != NON_ENCODED_QUALIFIERS) {
+                // We need to serialize the colRefs of index tables only in case of encoded column names.
+                ColumnReference indexTableColRef = e.getValue();
+                cRefBuilder =  ServerCachingProtos.ColumnReference.newBuilder();
+                cRefBuilder.setFamily(ByteStringer.wrap(indexTableColRef.getFamily()));
+                cRefBuilder.setQualifier(ByteStringer.wrap(indexTableColRef.getQualifier()));
+                builder.addIndexTableColRefForCoveredColumns(cRefBuilder.build());
+            }
+        }
+        builder.setIsLocalIndex(maintainer.isLocalIndex);
+        builder.setIndexTableName(ByteStringer.wrap(maintainer.indexTableName));
+        builder.setRowKeyOrderOptimizable(maintainer.rowKeyOrderOptimizable);
+        builder.setDataTableEmptyKeyValueColFamily(ByteStringer.wrap(maintainer.dataEmptyKeyValueCF));
+        ServerCachingProtos.ImmutableBytesWritable.Builder ibwBuilder = ServerCachingProtos.ImmutableBytesWritable.newBuilder();
+        ibwBuilder.setByteArray(ByteStringer.wrap(maintainer.emptyKeyValueCFPtr.get()));
+        ibwBuilder.setLength(maintainer.emptyKeyValueCFPtr.getLength());
+        ibwBuilder.setOffset(maintainer.emptyKeyValueCFPtr.getOffset());
+        builder.setEmptyKeyValueColFamily(ibwBuilder.build());
+        try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
+            DataOutput output = new DataOutputStream(stream);
+            for (Expression expression : maintainer.indexedExpressions) {
+                WritableUtils.writeVInt(output, ExpressionType.valueOf(expression).ordinal());
+                expression.write(output);
+            }
+            builder.setIndexedExpressions(ByteStringer.wrap(stream.toByteArray()));
+        }
+        try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) {
+            DataOutput output = new DataOutputStream(stream);
+            maintainer.rowKeyMetaData.write(output);
+            builder.setRowKeyMetadata(ByteStringer.wrap(stream.toByteArray()));
+        }
+        builder.setNumDataTableColFamilies(maintainer.nDataCFs);
+        builder.setIndexWalDisabled(maintainer.indexWALDisabled);
+        builder.setIndexRowKeyByteSize(maintainer.estimatedIndexRowKeyBytes);
+        builder.setImmutable(maintainer.immutableRows);
+        for (Pair<String, String> p : maintainer.indexedColumnsInfo) {
+            ServerCachingProtos.ColumnInfo.Builder ciBuilder = ServerCachingProtos.ColumnInfo.newBuilder();
+            if (p.getFirst() != null) {
+                ciBuilder.setFamilyName(p.getFirst());
+            }
+            ciBuilder.setColumnName(p.getSecond());
+            builder.addIndexedColumnInfo(ciBuilder.build());
+        }
+        builder.setEncodingScheme(maintainer.encodingScheme.getSerializedMetadataValue());
+        builder.setImmutableStorageScheme(maintainer.immutableStorageScheme.getSerializedMetadataValue());
+        return builder.build();
+    }
 
     public int getEstimatedByteSize() {
         int size = WritableUtils.getVIntSize(nIndexSaltBuckets);
@@ -1212,8 +1515,9 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             PDataType type = indexedColumnTypes.get(i);
             size += WritableUtils.getVIntSize(type.ordinal());
         }
-        size += WritableUtils.getVIntSize(coveredColumns.size());
-        for (ColumnReference ref : coveredColumns) {
+        Set<ColumnReference> dataTableColRefs = coveredColumnsMap.keySet();
+        size += WritableUtils.getVIntSize(dataTableColRefs.size());
+        for (ColumnReference ref : dataTableColRefs) {
             size += WritableUtils.getVIntSize(ref.getFamilyWritable().getSize());
             size += ref.getFamily().length;
             size += WritableUtils.getVIntSize(ref.getQualifierWritable().getSize());
@@ -1241,24 +1545,16 @@ 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())));
-        }
-
-        this.allColumns = Sets.newLinkedHashSetWithExpectedSize(indexedExpressions.size() + coveredColumns.size());
+        byte[] emptyKvQualifier = EncodedColumnsUtil.getEmptyKeyValueInfo(encodingScheme).getFirst();
+        dataEmptyKeyValueRef = new ColumnReference(emptyKeyValueCFPtr.copyBytesIfNecessary(), emptyKvQualifier);
+        this.allColumns = Sets.newLinkedHashSetWithExpectedSize(indexedExpressions.size() + coveredColumnsMap.size());
         // columns that are required to evaluate all expressions in indexedExpressions (not including columns in data row key)
         this.indexedColumns = Sets.newLinkedHashSetWithExpectedSize(indexedExpressions.size());
         for (Expression expression : indexedExpressions) {
         	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;
@@ -1267,7 +1563,7 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             expression.accept(visitor);
         }
         allColumns.addAll(indexedColumns);
-        allColumns.addAll(coveredColumns);
+        allColumns.addAll(coveredColumnsMap.keySet());
         
         int dataPkOffset = (isDataTableSalted ? 1 : 0) + (isMultiTenant ? 1 : 0);
         int nIndexPkColumns = getIndexPkColumnCount();
@@ -1311,12 +1607,21 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
     }
 
     private int getIndexPkColumnCount() {
-        return dataRowKeySchema.getFieldCount() + indexedExpressions.size() - (isDataTableSalted ? 1 : 0) - (isMultiTenant ? 1 : 0);
+        return getIndexPkColumnCount(dataRowKeySchema, indexedExpressions.size(), isDataTableSalted, isMultiTenant);
+    }
+    
+    private static int getIndexPkColumnCount(RowKeySchema rowKeySchema, int numIndexExpressions, boolean isDataTableSalted, boolean isMultiTenant) {
+        return rowKeySchema.getFieldCount() + numIndexExpressions - (isDataTableSalted ? 1 : 0) - (isMultiTenant ? 1 : 0);
     }
     
     private RowKeyMetaData newRowKeyMetaData() {
         return getIndexPkColumnCount() < 0xFF ? new ByteSizeRowKeyMetaData() : new IntSizedRowKeyMetaData();
     }
+    
+    private static RowKeyMetaData newRowKeyMetaData(IndexMaintainer i, RowKeySchema rowKeySchema, int numIndexExpressions, boolean isDataTableSalted, boolean isMultiTenant) {
+        int indexPkColumnCount = getIndexPkColumnCount(rowKeySchema, numIndexExpressions, isDataTableSalted, isMultiTenant);
+        return indexPkColumnCount < 0xFF ? i.new ByteSizeRowKeyMetaData() : i.new IntSizedRowKeyMetaData();
+    }
 
     private RowKeyMetaData newRowKeyMetaData(int capacity) {
         return capacity < 0xFF ? new ByteSizeRowKeyMetaData(capacity) : new IntSizedRowKeyMetaData(capacity);
@@ -1523,4 +1828,17 @@ public class IndexMaintainer implements Writable, Iterable<ColumnReference> {
             return udfParseNodes;
         }
     }
+    
+    public byte[] getEmptyKeyValueQualifier() {
+        return dataEmptyKeyValueRef.getQualifier();
+    }
+    
+    public Set<Pair<String, String>> getIndexedColumnInfo() {
+        return indexedColumnsInfo;
+    }
+    
+    public ImmutableStorageScheme getIndexStorageScheme() {
+        return immutableStorageScheme;
+    }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
index 05a01b9..fcabdfd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheClient.java
@@ -93,4 +93,5 @@ public class IndexMetaDataCacheClient {
          */
         return serverCache.addServerCache(ranges, ptr, txState, new IndexMetaDataCacheFactory(), cacheUsingTableRef);
     }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
index 56849fe..9edcafc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/IndexMetaDataCacheFactory.java
@@ -47,10 +47,10 @@ public class IndexMetaDataCacheFactory implements ServerCacheFactory {
     }
 
     @Override
-    public Closeable newCache (ImmutableBytesWritable cachePtr, byte[] txState, final MemoryChunk chunk) throws SQLException {
+    public Closeable newCache (ImmutableBytesWritable cachePtr, byte[] txState, final MemoryChunk chunk, boolean useProtoForIndexMaintainer) throws SQLException {
         // just use the standard keyvalue builder - this doesn't really need to be fast
         final List<IndexMaintainer> maintainers = 
-                IndexMaintainer.deserialize(cachePtr, GenericKeyValueBuilder.INSTANCE);
+                IndexMaintainer.deserialize(cachePtr, GenericKeyValueBuilder.INSTANCE, useProtoForIndexMaintainer);
         final Transaction txn;
         try {
             txn = txState.length!=0 ? MutationState.decodeTransaction(txState) : null;

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..4116101 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
@@ -38,6 +38,7 @@ import com.google.common.collect.Lists;
  */
 public class PhoenixIndexCodec extends BaseIndexCodec {
     public static final String INDEX_MD = "IdxMD";
+    public static final String INDEX_PROTO_MD = "IdxProtoMD";
     public static final String INDEX_UUID = "IdxUUID";
     public static final String INDEX_MAINTAINERS = "IndexMaintainers";
     private static KeyValueBuilder KV_BUILDER = GenericKeyValueBuilder.INSTANCE;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
index e515dbb..5da8be8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexFailurePolicy.java
@@ -226,4 +226,4 @@ public class PhoenixIndexFailurePolicy extends DelegateIndexFailurePolicy {
         }
         return indexTableNames;
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
index d22e957..39473dc 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/index/PhoenixIndexMetaData.java
@@ -47,10 +47,15 @@ public class PhoenixIndexMetaData implements IndexMetaData {
         if (attributes == null) { return IndexMetaDataCache.EMPTY_INDEX_META_DATA_CACHE; }
         byte[] uuid = attributes.get(PhoenixIndexCodec.INDEX_UUID);
         if (uuid == null) { return IndexMetaDataCache.EMPTY_INDEX_META_DATA_CACHE; }
-        byte[] md = attributes.get(PhoenixIndexCodec.INDEX_MD);
+        boolean useProto = false;
+        byte[] md = attributes.get(PhoenixIndexCodec.INDEX_PROTO_MD);
+        useProto = md != null;
+        if (md == null) {
+            md = attributes.get(PhoenixIndexCodec.INDEX_MD);
+        }
         byte[] txState = attributes.get(BaseScannerRegionObserver.TX_STATE);
         if (md != null) {
-            final List<IndexMaintainer> indexMaintainers = IndexMaintainer.deserialize(md);
+            final List<IndexMaintainer> indexMaintainers = IndexMaintainer.deserialize(md, useProto);
             final Transaction txn = MutationState.decodeTransaction(txState);
             return new IndexMetaDataCache() {
 


[17/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)


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

Branch: refs/heads/4.x-HBase-1.1
Commit: d6a82e29fce471bd47c42802ab4ac9548acf60ae
Parents: 03c2bc2
Author: Samarth <sa...@salesforce.com>
Authored: Sun Feb 26 22:22:03 2017 -0800
Committer: Samarth <sa...@salesforce.com>
Committed: Sun Feb 26 22:22:03 2017 -0800

----------------------------------------------------------------------
 .../phoenix/end2end/AggregateQueryIT.java       |   74 +-
 .../AlterMultiTenantTableWithViewsIT.java       |   25 +-
 .../apache/phoenix/end2end/AlterTableIT.java    |  491 +-
 .../phoenix/end2end/AlterTableWithViewsIT.java  |  133 +-
 .../org/apache/phoenix/end2end/ArrayIT.java     |   28 +
 .../org/apache/phoenix/end2end/BaseJoinIT.java  |    4 +-
 .../org/apache/phoenix/end2end/BaseQueryIT.java |   99 +-
 .../apache/phoenix/end2end/CaseStatementIT.java |   28 +-
 .../apache/phoenix/end2end/CastAndCoerceIT.java |   34 +-
 .../end2end/ClientTimeArithmeticQueryIT.java    |   76 +-
 .../end2end/ColumnEncodedBytesPropIT.java       |   95 +
 .../end2end/CountDistinctCompressionIT.java     |    2 +-
 .../apache/phoenix/end2end/CreateTableIT.java   |   57 +
 .../org/apache/phoenix/end2end/DateTimeIT.java  |    2 +-
 .../phoenix/end2end/DefaultColumnValueIT.java   |    1 +
 .../apache/phoenix/end2end/DerivedTableIT.java  |    2 +-
 .../apache/phoenix/end2end/DistinctCountIT.java |    4 +-
 .../apache/phoenix/end2end/DynamicColumnIT.java |   63 +
 .../phoenix/end2end/ExtendedQueryExecIT.java    |    8 +-
 .../apache/phoenix/end2end/FunkyNamesIT.java    |    2 +-
 .../org/apache/phoenix/end2end/GroupByIT.java   |  162 +-
 .../phoenix/end2end/ImmutableTablePropIT.java   |  130 -
 .../end2end/ImmutableTablePropertiesIT.java     |  189 +
 .../apache/phoenix/end2end/MutableQueryIT.java  |  424 ++
 .../phoenix/end2end/NativeHBaseTypesIT.java     |    2 +-
 .../org/apache/phoenix/end2end/NotQueryIT.java  |   28 +-
 .../org/apache/phoenix/end2end/OrderByIT.java   |    2 -
 .../apache/phoenix/end2end/PercentileIT.java    |    4 +-
 .../phoenix/end2end/PhoenixRuntimeIT.java       |    4 +-
 .../phoenix/end2end/PointInTimeQueryIT.java     |   78 +-
 .../phoenix/end2end/ProductMetricsIT.java       |    2 +-
 .../end2end/QueryDatabaseMetaDataIT.java        |   16 +-
 .../org/apache/phoenix/end2end/QueryIT.java     |  112 +-
 .../phoenix/end2end/ReadIsolationLevelIT.java   |    2 +-
 .../phoenix/end2end/RowValueConstructorIT.java  |   36 +-
 .../org/apache/phoenix/end2end/ScanQueryIT.java |   93 +-
 .../phoenix/end2end/StatsCollectorIT.java       |  124 +-
 .../apache/phoenix/end2end/StoreNullsIT.java    |  310 +-
 .../phoenix/end2end/StoreNullsPropIT.java       |   51 +
 ...SysTableNamespaceMappedStatsCollectorIT.java |    4 +-
 .../java/org/apache/phoenix/end2end/TopNIT.java |    6 +-
 .../apache/phoenix/end2end/UpsertSelectIT.java  |   10 +-
 .../apache/phoenix/end2end/UpsertValuesIT.java  |   51 +-
 .../phoenix/end2end/UserDefinedFunctionsIT.java |    3 +-
 .../phoenix/end2end/VariableLengthPKIT.java     |   38 +-
 .../phoenix/end2end/index/DropMetadataIT.java   |   13 +-
 .../phoenix/end2end/index/ImmutableIndexIT.java |   20 +-
 .../end2end/index/IndexExpressionIT.java        |   28 +-
 .../apache/phoenix/end2end/index/IndexIT.java   |   58 +-
 .../phoenix/end2end/index/IndexTestUtil.java    |   11 +-
 .../end2end/index/MutableIndexFailureIT.java    |    2 +
 .../phoenix/end2end/index/MutableIndexIT.java   |   21 +-
 .../phoenix/end2end/salted/SaltedTableIT.java   |    2 +-
 .../phoenix/tx/ParameterizedTransactionIT.java  |  518 ++
 .../org/apache/phoenix/tx/TransactionIT.java    |  589 +-
 .../org/apache/phoenix/tx/TxCheckpointIT.java   |   42 +-
 .../apache/phoenix/cache/ServerCacheClient.java |    2 +
 .../org/apache/phoenix/cache/TenantCache.java   |    2 +-
 .../apache/phoenix/cache/TenantCacheImpl.java   |    4 +-
 .../phoenix/compile/CreateTableCompiler.java    |   10 +-
 .../apache/phoenix/compile/DeleteCompiler.java  |    2 +-
 .../phoenix/compile/ExpressionCompiler.java     |   18 +-
 .../apache/phoenix/compile/FromCompiler.java    |   54 +-
 .../apache/phoenix/compile/JoinCompiler.java    |   15 +-
 .../phoenix/compile/ListJarsQueryPlan.java      |    6 +-
 .../apache/phoenix/compile/PostDDLCompiler.java |   11 +-
 .../compile/PostLocalIndexDDLCompiler.java      |    9 +-
 .../phoenix/compile/ProjectionCompiler.java     |   35 +-
 .../apache/phoenix/compile/TraceQueryPlan.java  |    4 +-
 .../compile/TupleProjectionCompiler.java        |   31 +-
 .../apache/phoenix/compile/UnionCompiler.java   |    7 +-
 .../apache/phoenix/compile/UpsertCompiler.java  |    6 +-
 .../apache/phoenix/compile/WhereCompiler.java   |   24 +-
 .../coprocessor/BaseScannerRegionObserver.java  |   59 +-
 .../GroupedAggregateRegionObserver.java         |   42 +-
 .../coprocessor/HashJoinRegionScanner.java      |   25 +-
 .../coprocessor/MetaDataEndpointImpl.java       |  237 +-
 .../phoenix/coprocessor/MetaDataProtocol.java   |    4 +-
 .../coprocessor/MetaDataRegionObserver.java     |    4 +-
 .../phoenix/coprocessor/ScanRegionObserver.java |   33 +-
 .../coprocessor/ServerCachingEndpointImpl.java  |    2 +-
 .../coprocessor/ServerCachingProtocol.java      |    2 +-
 .../UngroupedAggregateRegionObserver.java       |   65 +-
 .../coprocessor/generated/PTableProtos.java     | 1474 ++++-
 .../generated/ServerCachingProtos.java          | 5125 +++++++++++++++++-
 .../phoenix/exception/SQLExceptionCode.java     |    4 +
 .../apache/phoenix/execute/BaseQueryPlan.java   |   27 +-
 .../apache/phoenix/execute/CorrelatePlan.java   |    2 +-
 .../apache/phoenix/execute/MutationState.java   |   16 +-
 .../phoenix/execute/SortMergeJoinPlan.java      |    2 +-
 .../apache/phoenix/execute/TupleProjector.java  |   72 +-
 .../apache/phoenix/execute/UnnestArrayPlan.java |    3 +-
 .../expression/ArrayConstructorExpression.java  |   82 +-
 .../phoenix/expression/ExpressionType.java      |    5 +-
 .../expression/KeyValueColumnExpression.java    |   32 +-
 .../phoenix/expression/LiteralExpression.java   |   11 +-
 .../expression/ProjectedColumnExpression.java   |    1 +
 .../expression/SingleCellColumnExpression.java  |  182 +
 .../SingleCellConstructorExpression.java        |  102 +
 .../function/ArrayElemRefExpression.java        |    4 +-
 .../expression/function/ArrayIndexFunction.java |    4 +-
 .../expression/util/regex/JONIPattern.java      |   18 +-
 .../visitor/BaseExpressionVisitor.java          |    6 +
 .../visitor/CloneExpressionVisitor.java         |   12 +
 .../CloneNonDeterministicExpressionVisitor.java |    1 +
 .../expression/visitor/ExpressionVisitor.java   |    6 +
 .../StatelessTraverseAllExpressionVisitor.java  |   13 +-
 .../StatelessTraverseNoExpressionVisitor.java   |   13 +-
 .../phoenix/filter/ColumnProjectionFilter.java  |   24 +-
 ...EncodedQualifiersColumnProjectionFilter.java |  151 +
 .../MultiEncodedCQKeyValueComparisonFilter.java |  369 ++
 .../filter/MultiKeyValueComparisonFilter.java   |    6 +-
 .../SingleCQKeyValueComparisonFilter.java       |    3 +-
 .../filter/SingleKeyValueComparisonFilter.java  |    4 +-
 .../apache/phoenix/hbase/index/ValueGetter.java |    1 +
 .../hbase/index/util/KeyValueBuilder.java       |    1 +
 .../apache/phoenix/index/IndexMaintainer.java   |  530 +-
 .../phoenix/index/IndexMetaDataCacheClient.java |    1 +
 .../index/IndexMetaDataCacheFactory.java        |    4 +-
 .../phoenix/index/PhoenixIndexBuilder.java      |    2 +-
 .../apache/phoenix/index/PhoenixIndexCodec.java |    1 +
 .../index/PhoenixIndexFailurePolicy.java        |    2 +-
 .../phoenix/index/PhoenixIndexMetaData.java     |    9 +-
 .../index/PhoenixTransactionalIndexer.java      |   16 +-
 .../phoenix/iterate/BaseResultIterators.java    |  111 +-
 .../iterate/LookAheadResultIterator.java        |    2 +-
 .../phoenix/iterate/MappedByteBufferQueue.java  |    1 +
 .../phoenix/iterate/OrderedResultIterator.java  |    3 +-
 .../iterate/RegionScannerResultIterator.java    |   19 +-
 .../phoenix/jdbc/PhoenixDatabaseMetaData.java   |   14 +-
 .../apache/phoenix/jdbc/PhoenixResultSet.java   |    2 +-
 .../apache/phoenix/join/HashCacheFactory.java   |    2 +-
 .../mapreduce/FormatToBytesWritableMapper.java  |   21 +-
 .../mapreduce/FormatToKeyValueReducer.java      |   30 +-
 .../index/PhoenixIndexPartialBuildMapper.java   |    4 +-
 .../query/ConnectionQueryServicesImpl.java      |  135 +-
 .../query/ConnectionlessQueryServicesImpl.java  |    1 -
 .../apache/phoenix/query/QueryConstants.java    |   59 +-
 .../org/apache/phoenix/query/QueryServices.java |    4 +
 .../phoenix/query/QueryServicesOptions.java     |    6 +-
 .../org/apache/phoenix/schema/ColumnRef.java    |   16 +-
 .../phoenix/schema/ColumnValueDecoder.java      |   31 +
 .../phoenix/schema/ColumnValueEncoder.java      |   45 +
 .../apache/phoenix/schema/DelegateColumn.java   |    4 +
 .../apache/phoenix/schema/DelegateTable.java    |   24 +-
 .../apache/phoenix/schema/MetaDataClient.java   |  488 +-
 .../java/org/apache/phoenix/schema/PColumn.java |    4 +-
 .../apache/phoenix/schema/PColumnFamily.java    |   14 +-
 .../phoenix/schema/PColumnFamilyImpl.java       |   49 +-
 .../org/apache/phoenix/schema/PColumnImpl.java  |   29 +-
 .../apache/phoenix/schema/PMetaDataImpl.java    |    4 +-
 .../java/org/apache/phoenix/schema/PTable.java  |  435 +-
 .../org/apache/phoenix/schema/PTableImpl.java   |  480 +-
 .../org/apache/phoenix/schema/PTableKey.java    |    6 +-
 .../apache/phoenix/schema/ProjectedColumn.java  |   12 +-
 .../org/apache/phoenix/schema/SaltingUtil.java  |    2 +-
 .../apache/phoenix/schema/TableProperty.java    |   42 +
 .../apache/phoenix/schema/tuple/BaseTuple.java  |   39 +
 .../phoenix/schema/tuple/DelegateTuple.java     |    7 +
 .../tuple/EncodedColumnQualiferCellsList.java   |  581 ++
 .../schema/tuple/MultiKeyValueTuple.java        |    1 +
 .../tuple/PositionBasedMultiKeyValueTuple.java  |   90 +
 .../schema/tuple/PositionBasedResultTuple.java  |  125 +
 .../phoenix/schema/tuple/ResultTuple.java       |   20 +-
 .../org/apache/phoenix/schema/tuple/Tuple.java  |    4 +
 .../phoenix/schema/types/PArrayDataType.java    |  340 +-
 .../schema/types/PArrayDataTypeDecoder.java     |  102 +
 .../schema/types/PArrayDataTypeEncoder.java     |  170 +
 .../apache/phoenix/util/EncodedColumnsUtil.java |  205 +
 .../java/org/apache/phoenix/util/IndexUtil.java |   76 +-
 .../org/apache/phoenix/util/KeyValueUtil.java   |    2 -
 .../org/apache/phoenix/util/MetaDataUtil.java   |    5 +
 .../org/apache/phoenix/util/PhoenixRuntime.java |   12 +-
 .../org/apache/phoenix/util/ResultUtil.java     |   60 -
 .../java/org/apache/phoenix/util/ScanUtil.java  |   29 +-
 .../org/apache/phoenix/util/SchemaUtil.java     |   29 +-
 .../apache/phoenix/cache/TenantCacheTest.java   |    6 +-
 .../phoenix/compile/HavingCompilerTest.java     |    2 +-
 .../phoenix/compile/QueryCompilerTest.java      |   28 +-
 .../phoenix/compile/QueryOptimizerTest.java     |   51 +
 .../compile/SelectStatementRewriterTest.java    |   11 +-
 .../phoenix/compile/WhereCompilerTest.java      |   44 +-
 .../phoenix/execute/CorrelatePlanTest.java      |   12 +-
 .../execute/LiteralResultIteratorPlanTest.java  |   12 +-
 .../phoenix/execute/MutationStateTest.java      |    4 +-
 .../phoenix/execute/UnnestArrayPlanTest.java    |    8 +-
 .../ArrayConstructorExpressionTest.java         |   20 +-
 .../expression/ColumnExpressionTest.java        |   27 +-
 .../phoenix/index/IndexMaintainerTest.java      |    7 +-
 .../iterate/AggregateResultScannerTest.java     |    2 +-
 .../query/BaseConnectionlessQueryTest.java      |   18 +-
 .../java/org/apache/phoenix/query/BaseTest.java |   61 +-
 .../phoenix/query/ConnectionlessTest.java       |   18 +-
 .../EncodedColumnQualifierCellsListTest.java    |  608 +++
 .../schema/ImmutableStorageSchemeTest.java      |  182 +
 .../schema/types/PDataTypeForArraysTest.java    |   38 +-
 .../apache/phoenix/util/PhoenixRuntimeTest.java |    7 +-
 .../util/QualifierEncodingSchemeTest.java       |  119 +
 .../java/org/apache/phoenix/util/TestUtil.java  |   89 +-
 phoenix-protocol/src/main/PTable.proto          |    9 +
 .../src/main/ServerCachingService.proto         |   35 +
 pom.xml                                         |    2 +-
 202 files changed, 15812 insertions(+), 3071 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
index c689373..a205814 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AggregateQueryIT.java
@@ -17,12 +17,10 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.A_VALUE;
 import static org.apache.phoenix.util.TestUtil.B_VALUE;
 import static org.apache.phoenix.util.TestUtil.C_VALUE;
 import static org.apache.phoenix.util.TestUtil.E_VALUE;
-import static org.apache.phoenix.util.TestUtil.ROW3;
 import static org.apache.phoenix.util.TestUtil.ROW4;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
@@ -47,55 +45,13 @@ import org.junit.Test;
 
 public class AggregateQueryIT extends BaseQueryIT {
 
-    public AggregateQueryIT(String indexDDL) {
-        super(indexDDL);
-    }
-
-    @Test
-    public void testSumOverNullIntegerColumn() throws Exception {
-        String query = "SELECT sum(a_integer) FROM aTable a";
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.setAutoCommit(true);
-        conn.createStatement().execute("UPSERT INTO atable(organization_id,entity_id,a_integer) VALUES('" + getOrganizationId() + "','" + ROW3 + "',NULL)");
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
-        Connection conn1 = DriverManager.getConnection(getUrl(), props);
-        analyzeTable(conn1, "ATABLE");
-        conn1.close();
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 50));
-        conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(42, rs.getInt(1));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 70));
-        conn = DriverManager.getConnection(getUrl(), props);
-        conn.setAutoCommit(true);
-        conn.createStatement().execute("UPSERT INTO atable(organization_id,entity_id,a_integer) SELECT organization_id, entity_id, CAST(null AS integer) FROM atable");
-
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 90));
-        conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(0, rs.getInt(1));
-            assertTrue(rs.wasNull());
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
+    public AggregateQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
 
     @Test
     public void testGroupByPlusOne() throws Exception {
-        String query = "SELECT a_integer+1 FROM aTable WHERE organization_id=? and a_integer = 5 GROUP BY a_integer+1";
+        String query = "SELECT a_integer+1 FROM " + tableName + " WHERE organization_id=? and a_integer = 5 GROUP BY a_integer+1";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -114,7 +70,7 @@ public class AggregateQueryIT extends BaseQueryIT {
     @Test
     public void testSplitWithCachedMeta() throws Exception {
         // Tests that you don't get an ambiguous column exception when using the same alias as the column name
-        String query = "SELECT a_string, b_string, count(1) FROM atable WHERE organization_id=? and entity_id<=? GROUP BY a_string,b_string";
+        String query = "SELECT a_string, b_string, count(1) FROM " + tableName + " WHERE organization_id=? and entity_id<=? GROUP BY a_string,b_string";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -138,12 +94,12 @@ public class AggregateQueryIT extends BaseQueryIT {
             assertEquals(1, rs.getLong(3));
             assertFalse(rs.next());
             
-            byte[] tableName = Bytes.toBytes(ATABLE_NAME);
+            byte[] tableNameBytes = Bytes.toBytes(tableName);
             admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
-            HTable htable = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(tableName);
+            HTable htable = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(tableNameBytes);
             htable.clearRegionCache();
             int nRegions = htable.getRegionLocations().size();
-            admin.split(tableName, ByteUtil.concat(Bytes.toBytes(tenantId), Bytes.toBytes("00A" + Character.valueOf((char) ('3' + nextRunCount())) + ts))); // vary split point with test run
+            admin.split(tableNameBytes, ByteUtil.concat(Bytes.toBytes(tenantId), Bytes.toBytes("00A" + Character.valueOf((char) ('3' + nextRunCount())) + ts))); // vary split point with test run
             int retryCount = 0;
             do {
                 Thread.sleep(2000);
@@ -177,7 +133,7 @@ public class AggregateQueryIT extends BaseQueryIT {
     
     @Test
     public void testCountIsNull() throws Exception {
-        String query = "SELECT count(1) FROM aTable WHERE X_DECIMAL is null";
+        String query = "SELECT count(1) FROM " + tableName + " WHERE X_DECIMAL is null";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -195,7 +151,7 @@ public class AggregateQueryIT extends BaseQueryIT {
     
     @Test
     public void testCountWithNoScanRanges() throws Exception {
-        String query = "SELECT count(1) FROM aTable WHERE organization_id = 'not_existing_organization_id'";
+        String query = "SELECT count(1) FROM " + tableName + " WHERE organization_id = 'not_existing_organization_id'";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
 		Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -205,19 +161,19 @@ public class AggregateQueryIT extends BaseQueryIT {
 			assertTrue(rs.next());
 			assertEquals(0, rs.getLong(1));
 			assertFalse(rs.next());
-			query = "SELECT count(1) FROM aTable WHERE organization_id = 'not_existing_organization_id' having count(*)>0";
+			query = "SELECT count(1) FROM " + tableName + " WHERE organization_id = 'not_existing_organization_id' having count(*)>0";
 			rs = conn.prepareStatement(query).executeQuery();
 			assertFalse(rs.next());
-			query = "SELECT count(1) FROM aTable WHERE organization_id = 'not_existing_organization_id' limit 1 offset 1";
+			query = "SELECT count(1) FROM " + tableName + " WHERE organization_id = 'not_existing_organization_id' limit 1 offset 1";
 			rs = conn.prepareStatement(query).executeQuery();
 			assertFalse(rs.next());
-			query = "SELECT count(1),123 FROM aTable WHERE organization_id = 'not_existing_organization_id'";
+			query = "SELECT count(1),123 FROM " + tableName + " WHERE organization_id = 'not_existing_organization_id'";
 			rs = conn.prepareStatement(query).executeQuery();
 			assertTrue(rs.next());
 			assertEquals(0, rs.getLong(1));
 			assertEquals("123", rs.getString(2));
 			assertFalse(rs.next());
-			query = "SELECT count(1),sum(x_decimal) FROM aTable WHERE organization_id = 'not_existing_organization_id'";
+			query = "SELECT count(1),sum(x_decimal) FROM " + tableName + " WHERE organization_id = 'not_existing_organization_id'";
 			rs = conn.prepareStatement(query).executeQuery();
 			assertTrue(rs.next());
 			assertEquals(0, rs.getLong(1));
@@ -230,7 +186,7 @@ public class AggregateQueryIT extends BaseQueryIT {
 
     @Test
     public void testCountIsNotNull() throws Exception {
-        String query = "SELECT count(1) FROM aTable WHERE X_DECIMAL is not null";
+        String query = "SELECT count(1) FROM " + tableName + " WHERE X_DECIMAL is not null";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -250,7 +206,7 @@ public class AggregateQueryIT extends BaseQueryIT {
      */
     @Test
     public void testInFilterOnKey() throws Exception {
-        String query = "SELECT count(entity_id) FROM ATABLE WHERE organization_id IN (?,?)";
+        String query = "SELECT count(entity_id) FROM " + tableName + " WHERE organization_id IN (?,?)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
index d6f3a7f..7b4ff68 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterMultiTenantTableWithViewsIT.java
@@ -476,14 +476,15 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
             // For a diverged view, only base table's pk column will be added and that too at the end.
             assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
             
-            // Add existing column VIEW_COL2 to the base table
-            alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
-            conn.createStatement().execute(alterBaseTable);
-            
-            // For the non-diverged view, adding the column VIEW_COL2 will end up changing its ordinal position in the view.
-            assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 2, 9, 8, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL2", "VIEW_COL1");
-            // For the diverged view, adding the column VIEW_COL2 will not change its ordinal position in the view. It also won't change the base column count or the sequence number
-            assertTableDefinition(conn, divergedView, PTableType.VIEW, baseTable, 2, 7, DIVERGED_VIEW_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2", "PK2");
+            // Adding existing column VIEW_COL2 to the base table isn't allowed.
+            try {
+                alterBaseTable = "ALTER TABLE " + baseTable + " ADD VIEW_COL2 CHAR(256)";
+                conn.createStatement().execute(alterBaseTable);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals("Unexpected exception", SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            }
         }
     }
     
@@ -500,13 +501,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " ADD KV VARCHAR, PK2 VARCHAR PRIMARY KEY";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 7, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 9, 7, "TENANT_ID", "PK1", "V1", "V2", "V3", "KV", "PK2", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the both columns were added to view1
@@ -530,13 +531,13 @@ public class AlterMultiTenantTableWithViewsIT extends ParallelStatsDisabledIT {
                 tenant1Conn.createStatement().execute(view1DDL);
             }
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 0, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 5, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V2", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 0, 7, 5, "TENANT_ID", "PK1", "V1", "V2", "V3", "VIEW_COL1", "VIEW_COL2");
 
             String alterBaseTable = "ALTER TABLE " + baseTable + " DROP COLUMN V2";
             conn.createStatement().execute(alterBaseTable);
 
-            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 1, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
+            assertTableDefinition(conn, baseTable, PTableType.TABLE, null, 2, 4, BASE_TABLE_BASE_COLUMN_COUNT, "TENANT_ID", "PK1", "V1", "V3");
             assertTableDefinition(conn, view1, PTableType.VIEW, baseTable, 1, 6, 4, "TENANT_ID", "PK1", "V1", "V3", "VIEW_COL1", "VIEW_COL2");
 
             // verify that the dropped columns aren't visible

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 fee7316..d13b8d2 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;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
+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;
@@ -35,6 +44,8 @@ import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
@@ -53,6 +64,8 @@ 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.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.util.IndexUtil;
@@ -60,6 +73,9 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  *
@@ -72,14 +88,28 @@ import org.junit.Test;
  * or at the end of test class.
  *
  */
+@RunWith(Parameterized.class)
 public class AlterTableIT extends ParallelStatsDisabledIT {
     private String schemaName;
     private String dataTableName;
     private String indexTableName;
     private String localIndexTableName;
+    private String viewName;
     private String dataTableFullName;
     private String indexTableFullName;
     private String localIndexTableFullName;
+    private String tableDDLOptions;
+    private final boolean columnEncoded;
+    
+    public AlterTableIT(boolean columnEncoded) {
+        this.columnEncoded = columnEncoded;
+        this.tableDDLOptions = columnEncoded ? "" : "COLUMN_ENCODED_BYTES=0";
+    }
+    
+    @Parameters(name="AlterTableIT_columnEncoded={0}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean> data() {
+        return Arrays.asList( false, true);
+    }
 
     @Before
     public void setupTableNames() throws Exception {
@@ -90,6 +120,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         dataTableFullName = SchemaUtil.getTableName(schemaName, dataTableName);
         indexTableFullName = SchemaUtil.getTableName(schemaName, indexTableName);
         localIndexTableFullName = SchemaUtil.getTableName(schemaName, localIndexTableName);
+        viewName = generateUniqueName();
     }
 
     @Test
@@ -101,7 +132,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         try {
             String ddl = "CREATE TABLE  " + dataTableFullName +
                     "  (a_string varchar not null, a_binary varbinary not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary))\n";
+                    "  CONSTRAINT pk PRIMARY KEY (a_string, a_binary)) " + tableDDLOptions;
             createTestTable(getUrl(), ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " ADD b_string VARCHAR NULL PRIMARY KEY";
@@ -142,7 +173,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         try {
             String ddl = "CREATE TABLE " + dataTableFullName +
                     "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             String dml = "UPSERT INTO " + dataTableFullName + " VALUES(?)";
@@ -209,7 +240,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
         String ddl = "CREATE TABLE  " + dataTableFullName +
                 "  (a_string varchar not null, col1 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " ADD CF.col2 integer CF.IN_MEMORY=true");
@@ -231,8 +262,31 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         assertEquals(exists, rs.next());
     }
 
+    
     @Test
-    public void testDropIndexedColumn() throws Exception {
+    public void testDropIndexedColumnImmutableIndex() throws Exception {
+        helpTestDropIndexedColumn(true);
+    }
+    
+    @Test
+    public void testDropIndexedColumnMutableIndex() throws Exception {
+        helpTestDropIndexedColumn(false);
+    }
+    
+    private String generateDDLOptions(String options) {
+        StringBuilder sb = new StringBuilder();
+        if (!options.isEmpty()) {
+            sb.append(options);
+        }
+        if (!tableDDLOptions.isEmpty()) {
+            if (sb.length()!=0)
+                sb.append(",");
+            sb.append(tableDDLOptions);
+        }
+        return sb.toString();
+    }
+    
+    private void helpTestDropIndexedColumn(boolean immutable) throws Exception {
         String query;
         ResultSet rs;
         PreparedStatement stmt;
@@ -244,7 +298,9 @@ 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) "
+              + generateDDLOptions(immutable ? "IMMUTABLE_ROWS = true" : "")
+              + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ""));
         query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
@@ -304,7 +360,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, v3 VARCHAR)");
+              + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR, v3 VARCHAR) " +  tableDDLOptions);
         String dataTableQuery = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(dataTableQuery);
         assertFalse(rs.next());
@@ -419,7 +475,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) " + tableDDLOptions);
         query = "SELECT * FROM " + dataTableFullName;
         rs = conn.createStatement().executeQuery(query);
         assertFalse(rs.next());
@@ -515,7 +571,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             		"        B.M_TEXT VARCHAR\n" +
             		"        CONSTRAINT ROWKEY PRIMARY KEY\n" +
             		"(SENDER_ID,RECIPIENT_ID,M_TIMESTAMP DESC,ROW_ID))\n" +
-            		"SALT_BUCKETS=4";
+            		generateDDLOptions("SALT_BUCKETS=4");
             conn.createStatement().execute(ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " SET IMMUTABLE_ROWS=true";
@@ -549,7 +605,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     "        B.M_TEXT VARCHAR\n" +
                     "        CONSTRAINT ROWKEY PRIMARY KEY\n" +
                     "(SENDER_ID,RECIPIENT_ID,M_TIMESTAMP DESC,ROW_ID))\n" +
-                    "SALT_BUCKETS=4";
+                    generateDDLOptions("SALT_BUCKETS=4");
             conn.createStatement().execute(ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " DROP COLUMN B.JSON";
@@ -572,7 +628,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         try {
             String ddl = "CREATE TABLE " + dataTableFullName +
                     "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             String dml = "UPSERT INTO " + dataTableFullName + " VALUES(?)";
@@ -679,7 +735,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         conn.setAutoCommit(false);
         try {
             String ddl = "CREATE TABLE " + dataTableFullName + " " + "  (a_string varchar not null, col1 integer, cf1.col2 integer"
-                    + "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             ddl = "ALTER TABLE " + dataTableFullName + " DROP COLUMN col1";
@@ -702,7 +758,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
             conn.setAutoCommit(false);
             try {
                 String ddl = "CREATE TABLE " + dataTableFullName + " " + "  (a_string varchar not null, col1 integer, cf1.col2 integer"
-                        + "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                        + "  CONSTRAINT pk PRIMARY KEY (a_string)) " +  tableDDLOptions;
                 stmt = conn.prepareStatement(ddl);
                 stmt.execute();
             } finally {
@@ -737,7 +793,9 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true, disable_wal=true ");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) "
+                                    + generateDDLOptions("immutable_rows=true, disable_wal=true"
+                                    + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : "")));
 
             Connection conn2 = DriverManager.getConnection(getUrl(), props);
             String query = "SELECT * FROM " + dataTableFullName;
@@ -769,7 +827,9 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string))" 
+                                    + generateDDLOptions("immutable_rows=true"
+                                    + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : "")));
 
             Connection conn2 = DriverManager.getConnection(getUrl(), props);
             String query = "SELECT * FROM " + dataTableFullName;
@@ -802,7 +862,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string))");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions);
 
             Connection conn2 = DriverManager.getConnection(getUrl(), props);
             String query = "SELECT * FROM " + dataTableFullName;
@@ -840,7 +900,9 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     .execute(
                             "CREATE TABLE " + dataTableFullName
                                     + "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true , SALT_BUCKETS=3 ");
+                                    + "  CONSTRAINT pk PRIMARY KEY (a_string)) " 
+                                    + generateDDLOptions("immutable_rows=true , SALT_BUCKETS=3 "
+                                    + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : "")));
 
             String query = "SELECT * FROM " + dataTableFullName;
             ResultSet rs = conn.createStatement().executeQuery(query);
@@ -929,7 +991,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
         // here we insert into the orig schema with one column
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
-        conn1.createStatement().execute("create table " + dataTableFullName + "(id VARCHAR PRIMARY KEY, field1 BIGINT)");
+        conn1.createStatement().execute("create table " + dataTableFullName + "(id VARCHAR PRIMARY KEY, field1 BIGINT) " + tableDDLOptions);
         PreparedStatement stmtInsert1 = conn1.prepareStatement("upsert into " + dataTableFullName + " (id, field1) values ( ?, ?)");
         stmtInsert1.setString(1, "key1");
         stmtInsert1.setLong(2, 1L);
@@ -961,7 +1023,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2))";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) "  + tableDDLOptions;
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " ADD STRING VARCHAR, STRING_DATA_TYPES VARCHAR";
@@ -978,7 +1040,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         String ddl = "CREATE TABLE " + dataTableFullName + " (\n"
                 +"ID VARCHAR(15) PRIMARY KEY,\n"
-                +"COL1 BIGINT)";
+                +"COL1 BIGINT) " + tableDDLOptions;
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         conn1.createStatement().execute("CREATE INDEX " + indexTableName + " ON " + dataTableFullName + "(COL1)");
@@ -1035,7 +1097,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 + "COL1 BIGINT,"
                 + "COL2 BIGINT,"
                 + "COL3 BIGINT,"
-                + "COL4 BIGINT)";
+                + "COL4 BIGINT) " + tableDDLOptions;
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         conn1.createStatement().execute("CREATE INDEX " + indexTableName + " ON " + dataTableFullName + "(COL1) INCLUDE (COL2,COL3,COL4)");
@@ -1091,7 +1153,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET REPLICATION_SCOPE=1";
@@ -1114,7 +1176,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET COMPACTION_ENABLED=FALSE";
@@ -1136,7 +1198,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET COMPACTION_ENABLED = FALSE, REPLICATION_SCOPE = 1";
@@ -1160,7 +1222,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CF1.CREATION_TIME BIGINT,\n"
                 +"CF2.LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) IMMUTABLE_ROWS=true";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("IMMUTABLE_ROWS=true"
+                + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ""));
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         assertImmutableRows(conn, dataTableFullName, true);
@@ -1202,7 +1265,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.COMPACTION_ENABLED = FALSE";
@@ -1223,7 +1286,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.DISABLE_WAL = TRUE";
@@ -1244,7 +1307,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"CF.LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.TTL = 86400";
@@ -1265,7 +1328,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET CF.REPLICATION_SCOPE = 1";
@@ -1286,7 +1349,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions(" SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         ddl = "ALTER TABLE " + dataTableFullName + " SET DEFAULT_COLUMN_FAMILY = 'A'";
@@ -1307,19 +1370,19 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
-        ddl = "CREATE VIEW v AS SELECT * FROM " + dataTableFullName + " WHERE CREATION_TIME = 1";
+        ddl = "CREATE VIEW " + viewName + "  AS SELECT * FROM " + dataTableFullName + " WHERE CREATION_TIME = 1";
         conn1.createStatement().execute(ddl);
-        ddl = "ALTER VIEW v SET REPLICATION_SCOPE = 1";
+        ddl = "ALTER VIEW " + viewName + " SET REPLICATION_SCOPE = 1";
         try {
             conn1.createStatement().execute(ddl);
             fail();
         } catch (SQLException e) {
             assertEquals(SQLExceptionCode.VIEW_WITH_PROPERTIES.getErrorCode(), e.getErrorCode());
         }
-        ddl = "ALTER VIEW v SET COMPACTION_ENABLED = FALSE";
+        ddl = "ALTER VIEW " + viewName + " SET COMPACTION_ENABLED = FALSE";
         try {
             conn1.createStatement().execute(ddl);
             fail();
@@ -1337,7 +1400,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) SALT_BUCKETS = 8";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("SALT_BUCKETS = 8");
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
         conn1.createStatement().execute(ddl);
         String viewFullName = SchemaUtil.getTableName(schemaName, generateUniqueName());
@@ -1372,7 +1435,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                 +"CREATED_DATE DATE,\n"
                 +"CREATION_TIME BIGINT,\n"
                 +"CF.LAST_USED DATE,\n"
-                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) IMMUTABLE_ROWS=true, DEFAULT_COLUMN_FAMILY = 'XYZ'";
+                +"CONSTRAINT PK PRIMARY KEY (ID1, ID2)) " + generateDDLOptions("IMMUTABLE_ROWS=true, DEFAULT_COLUMN_FAMILY = 'XYZ'"
+                + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ""));
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute(ddl);
         assertImmutableRows(conn, dataTableFullName, true);
@@ -1406,7 +1470,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName
                 +
                 "  (a_string varchar not null, col1 integer, CF.col2 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute(
@@ -1432,7 +1496,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName + " "
                 +
                 "  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement()
@@ -1466,7 +1530,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName + " "
                 +
                 "  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ'");
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement()
@@ -1500,7 +1564,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName + " "
                 +
     			"  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-    			"  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+    			"  CONSTRAINT pk PRIMARY KEY (a_string)) "+ generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ'");
     	try {
     		conn.createStatement().execute(ddl);
     		try {
@@ -1523,7 +1587,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName
                 +
                 "  (a_string varchar not null, col1 integer, CF1.col2 integer, CF2.col3 integer" +
-                "  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+                "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ' ");
         try {
             conn.createStatement().execute(ddl);
             conn.createStatement()
@@ -1560,7 +1624,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         String ddl = "CREATE TABLE " + dataTableFullName
                 +
     			"  (a_string varchar not null, col1 integer, CF1.col2 integer" +
-    			"  CONSTRAINT pk PRIMARY KEY (a_string)) DEFAULT_COLUMN_FAMILY = 'XYZ'\n";
+    			"  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("DEFAULT_COLUMN_FAMILY = 'XYZ'");
     	try {
             conn.createStatement().execute(ddl);
             conn.createStatement().execute(
@@ -1590,7 +1654,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     	conn.setAutoCommit(false);
         String ddl = "CREATE TABLE " + dataTableFullName +
     			"  (a_string varchar not null, col1 integer" +
-    			"  CONSTRAINT pk PRIMARY KEY (a_string))\n";
+    			"  CONSTRAINT pk PRIMARY KEY (a_string)) " + tableDDLOptions;
     	try {
     		conn.createStatement().execute(ddl);
             conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " ADD col2 integer IN_MEMORY=true");
@@ -1618,7 +1682,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                             "CREATE TABLE "
                                     + dataTableFullName
     						+ "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true , SALT_BUCKETS=3 ");
+    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("immutable_rows=true , SALT_BUCKETS=3 "
+    						+ (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ""))); 
 
             String ddl = "Alter table " + dataTableFullName + " add cf3.col5 integer, cf4.col6 integer in_memory=true";
     		conn.createStatement().execute(ddl);
@@ -1656,7 +1721,8 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                             "CREATE TABLE "
                                     + dataTableFullName
     						+ "  (a_string varchar not null, col1 integer, cf1.col2 integer, col3 integer , cf2.col4 integer "
-    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) immutable_rows=true , SALT_BUCKETS=3 ");
+    						+ "  CONSTRAINT pk PRIMARY KEY (a_string)) " + generateDDLOptions("immutable_rows=true , SALT_BUCKETS=3 "
+    						+ (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : "")));    
 
             String ddl = "Alter table " + dataTableFullName + " add cf1.col5 integer in_memory=true";
     		conn.createStatement().execute(ddl);
@@ -1686,7 +1752,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
     	try {
             String ddl = "CREATE TABLE " + dataTableFullName
-                    + " (pk char(2) not null primary key, col1 integer, b.col1 integer) SPLIT ON ('EA','EZ')";
+                    + " (pk char(2) not null primary key, col1 integer, b.col1 integer) " + tableDDLOptions + " SPLIT ON ('EA','EZ') ";
     		conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " add b.col2 varchar ttl=30";
     		conn.createStatement().execute(ddl);
@@ -1706,7 +1772,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
     	try {
             String ddl = "CREATE TABLE " + dataTableFullName
-                    + " (pk char(2) not null primary key) TTL=100 SPLIT ON ('EA','EZ')";
+                    + " (pk char(2) not null primary key) " + generateDDLOptions("TTL=100") + " SPLIT ON ('EA','EZ')";
     		conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " add col1 varchar ttl=30";
     		conn.createStatement().execute(ddl);
@@ -1736,7 +1802,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             try (HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
                 HTableDescriptor tableDesc = admin.getTableDescriptor(Bytes.toBytes(dataTableFullName));
@@ -1771,7 +1837,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " SET IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1799,7 +1865,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " SET IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1827,7 +1893,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD COL3 INTEGER IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1855,7 +1921,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD NEWCF.COL3 INTEGER IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1885,7 +1951,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         		    + " col1 integer NOT NULL,"
         		    + " col2 bigint NOT NULL,"
         		    + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-        		    + " ) TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'";
+        		    + " ) " + generateDDLOptions("TTL=86400, SALT_BUCKETS = 4, DEFAULT_COLUMN_FAMILY='XYZ'");
             conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD NEWCF.COL3 INTEGER IN_MEMORY=true";
     		conn.createStatement().execute(ddl);
@@ -1949,7 +2015,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     				+ " col1 integer NOT NULL,"
     				+ " col2 bigint NOT NULL,"
     				+ " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-    				+ " )";
+    				+ " ) " +  tableDDLOptions;
     		conn.createStatement().execute(ddl);
             ddl = "ALTER TABLE " + dataTableFullName + " ADD NEWCF.COL3 INTEGER NEWCF.UNKNOWN_PROP='ABC'";
     		try {
@@ -1975,7 +2041,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
         Connection conn = DriverManager.getConnection(getUrl(), props);
 
         Statement stmt = conn.createStatement();
-        stmt.execute("CREATE TABLE " + dataTableFullName + " (id SMALLINT PRIMARY KEY, name VARCHAR)");
+        stmt.execute("CREATE TABLE " + dataTableFullName + " (id SMALLINT PRIMARY KEY, name VARCHAR) "+tableDDLOptions);
 
         ResultSet rs = stmt.executeQuery("SELECT STORE_NULLS FROM \"SYSTEM\".\"CATALOG\" " +
  "WHERE table_name = '"
@@ -2007,7 +2073,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     + " k2 integer NOT NULL,"
                     + " col1 bigint,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (k1, k2)"
-                    + " )";
+                    + " ) "+tableDDLOptions;
             conn.createStatement().execute(ddl);
 
             // set HTableProperty when adding a pk column should fail
@@ -2081,7 +2147,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
                     + " col1 integer NOT NULL,"
                     + " col2 bigint NOT NULL,"
                     + " CONSTRAINT NAME_PK PRIMARY KEY (id, col1, col2)"
-                    + " )";
+                    + " ) "+tableDDLOptions;
             conn.createStatement().execute(ddl);
             asssertIsWALDisabled(conn, dataTableFullName, false);
             
@@ -2118,12 +2184,12 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     @Test
     public void testDeclaringColumnAsRowTimestamp() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) ");
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 DATE NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1 ROW_TIMESTAMP, PK2)) " + tableDDLOptions);
             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.getColumnForColumnName("PK1").isRowTimestamp());
+            assertFalse(table.getColumnForColumnName("PK2").isRowTimestamp());
             assertIsRowTimestampSet(schemaName, dataTableName, "PK1");
             
             String dataTableName2 = BaseTest.generateUniqueName();
@@ -2131,18 +2197,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.getColumnForColumnName("PK1").isRowTimestamp());
+            assertTrue(table.getColumnForColumnName("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.getColumnForColumnName("PK2"));
             // Assert that the column shows up as row time stamp in the cache.
-            assertTrue(indexTable.getColumn(indexColName).isRowTimestamp());
+            assertTrue(indexTable.getColumnForColumnName(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
@@ -2177,7 +2242,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
     @Test
     public void testAddingRowTimestampColumnNotAllowedViaAlterTable() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 VARCHAR NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2)) ");
+            conn.createStatement().execute("CREATE TABLE " + dataTableFullName + " (PK1 VARCHAR NOT NULL, PK2 VARCHAR NOT NULL, KV1 VARCHAR CONSTRAINT PK PRIMARY KEY(PK1, PK2)) " + tableDDLOptions);
             // adding a new pk column that is also row_timestamp is not allowed
             try {
                 conn.createStatement().execute("ALTER TABLE " + dataTableFullName + " ADD PK3 DATE PRIMARY KEY ROW_TIMESTAMP");
@@ -2195,7 +2260,7 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 		try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
 			// creating a transactional table should fail if transactions are disabled
 			try {
-				conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR) TRANSACTIONAL=true");
+				conn.createStatement().execute("CREATE TABLE " + dataTableFullName + "(k INTEGER PRIMARY KEY, v VARCHAR) " + generateDDLOptions("TRANSACTIONAL=true"));
 				fail();
 			} catch (SQLException e) {
 				assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_IF_TXNS_DISABLED.getErrorCode(), e.getErrorCode());
@@ -2211,5 +2276,299 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 		}
 	}
 	
+	@Test
+	public void testMetadataForImmutableTable() 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)"
+	                + " ) " + generateDDLOptions("IMMUTABLE_ROWS = true"
+	                + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME="+ PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : "")));
+	        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( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1) : null, cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
+	        
+	        // assert that the server side metadata is updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+	        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( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 2) : null, baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        assertEquals( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1) : null, baseTable.getEncodedCQCounter().getNextQualifier("A"));
+	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        
+	        // assert that the server side metadata for the base table and the view is also updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+	        assertEncodedCQCounter("A", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+	        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 + (columnEncoded ? 1 : 0));
+	        assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM);
+	    }
+	}
+	
+	@Test
+	public void testMetadataForMutableTable() 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)"
+	                + " ) " + tableDDLOptions);
+	        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( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 1) : null, cqCounter.getNextQualifier(QueryConstants.DEFAULT_COLUMN_FAMILY));
+
+
+	        // assert that the server side metadata is updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+	        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(columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3) : null, baseTable.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+	        assertNull("A view should always have the null cq counter", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+
+	        // assert that the server side metadata for the base table and the view is also updated correctly.
+	        assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+	        assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+	        assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+	        assertSequenceNumber(schemaName, baseTableName, initBaseTableSeqNumber + (columnEncoded ? 1 : 0));
+	        assertSequenceNumber(schemaName, viewName, PTable.INITIAL_SEQ_NUM);
+	    }
+	}
+	
+	@Test
+    public void testAddingColumnsToTablesAndViews() throws Exception {
+        String schemaName = generateUniqueName();
+        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)"
+                    + " ) " + tableDDLOptions);
+            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( columnEncoded ?(Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 3) : null, encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert that the server side metadata is updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+            
+            // assert that the server side metadata for columns is updated correctly.
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL4", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL5", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+            assertEncodedCQValue("B", "COL6", schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 2);
+            long baseTableSeqNumBeforeAddingChildCols = initBaseTableSeqNumber + 1;
+            assertSequenceNumber(schemaName, baseTableName, baseTableSeqNumBeforeAddingChildCols);
+
+            // Create a view
+            String viewDDL = "CREATE VIEW " + fullViewName + " ( VIEW_COL1 INTEGER, A.VIEW_COL2 VARCHAR ) AS SELECT * FROM " + fullTableName;
+            conn.createStatement().execute(viewDDL);
+            
+            // assert that the server side metadata is updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5);
+            
+            // assert that the server side metadata for columns is updated correctly.
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+            assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 4);
+            // for encoded columns creating a view that adds its own columns should increment the base table's sequence number too.
+            assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 2 : baseTableSeqNumBeforeAddingChildCols );
+
+            // 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( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 8) : null, encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert client cache for view
+            PTable view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+            encodedCqCounter = view.getEncodedCQCounter();
+            assertNull("A view should always have the column qualifier counter as null", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert that the server side metadata for the base table and the view is also updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 8);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL1", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 3);
+            assertEncodedCQValue("A", "VIEW_COL2", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 4);
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "VIEW_COL3", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 5);
+            assertEncodedCQValue("A", "VIEW_COL4", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 6);
+            assertEncodedCQValue("B", "VIEW_COL5", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 7);
+            // adding a column to the should increment the base table's sequence number too since we update the cq counters for column families.
+            assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 3 : baseTableSeqNumBeforeAddingChildCols );
+            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( columnEncoded ? (Integer)(ENCODED_CQ_COUNTER_INITIAL_VALUE + 10) : null, encodedCqCounter.getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert client cache for view
+            view = phxConn.getTable(new PTableKey(phxConn.getTenantId(), fullViewName));
+            encodedCqCounter = view.getEncodedCQCounter();
+            assertNull("A view should always have the column qualifier counter as null", view.getEncodedCQCounter().getNextQualifier(DEFAULT_COLUMN_FAMILY));
+            
+            // assert that the server side metadata for the base table and the view is also updated correctly.
+            assertEncodedCQCounter(DEFAULT_COLUMN_FAMILY, schemaName, baseTableName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 10));
+            assertEncodedCQValue(DEFAULT_COLUMN_FAMILY, "COL10", schemaName, viewName, (ENCODED_CQ_COUNTER_INITIAL_VALUE + 8));
+            assertEncodedCQValue("A", "COL11", schemaName, viewName, ENCODED_CQ_COUNTER_INITIAL_VALUE + 9);
+            assertSequenceNumber(schemaName, baseTableName, columnEncoded ? initBaseTableSeqNumber + 4 : initBaseTableSeqNumber + 2 );
+            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 " + COLUMN_QUALIFIER + " FROM SYSTEM.CATALOG WHERE " + TABLE_SCHEM + " = ? AND " + TABLE_NAME
+                + " = ? " + " AND " + COLUMN_FAMILY + " = ?" + " AND " + COLUMN_NAME  + " = ?" + " AND " + COLUMN_QUALIFIER  + " 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);
+            stmt.setString(4, columnName);
+            ResultSet rs = stmt.executeQuery();
+            assertTrue(rs.next());
+            if (columnEncoded) {
+                assertTrue(Bytes.equals(QualifierEncodingScheme.TWO_BYTE_QUALIFIERS.encode(expectedValue), rs.getBytes(1)));
+            } else {
+                assertTrue(Bytes.equals(columnName.getBytes(), rs.getBytes(1)));
+            }
+            assertFalse(rs.next());
+        }
+    }
+    
+    private void assertEncodedCQCounter(String columnFamily, String schemaName, String tableName, int expectedValue) 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 (columnEncoded) {
+                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());
+        }
+    }
+    
+    @Test
+    public void testAlterImmutableRowsPropertyForOneCellPerKeyValueColumnStorageScheme() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String ddl = "CREATE TABLE " + dataTableFullName + " (\n"
+                +"ID VARCHAR(15) NOT NULL,\n"
+                +"CREATED_DATE DATE,\n"
+                +"CREATION_TIME BIGINT,\n"
+                +"CONSTRAINT PK PRIMARY KEY (ID)) " + tableDDLOptions;
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute(ddl);
+        assertImmutableRows(conn, dataTableFullName, false);
+        ddl = "ALTER TABLE " + dataTableFullName + " SET IMMUTABLE_ROWS = true";
+        conn.createStatement().execute(ddl);
+        assertImmutableRows(conn, dataTableFullName, true);
+    }
+    
+    @Test
+    public void testAlterImmutableRowsPropertyForOneCellPerColumnFamilyStorageScheme() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String ddl = "CREATE TABLE " + dataTableFullName + " (\n"
+                +"ID VARCHAR(15) NOT NULL,\n"
+                +"CREATED_DATE DATE,\n"
+                +"CREATION_TIME BIGINT,\n"
+                +"CONSTRAINT PK PRIMARY KEY (ID)) " + generateDDLOptions("COLUMN_ENCODED_BYTES=4, IMMUTABLE_ROWS=true"
+                + (!columnEncoded ? ",IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ""));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.createStatement().execute(ddl);
+        assertImmutableRows(conn, dataTableFullName, true);
+        try {
+	        ddl = "ALTER TABLE " + dataTableFullName + " SET IMMUTABLE_ROWS = false";
+	        conn.createStatement().execute(ddl);
+	        if (columnEncoded) {
+	            fail();
+	        }
+        }
+        catch(SQLException e) {
+        	assertEquals(SQLExceptionCode.CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY.getErrorCode(), e.getErrorCode());
+        }
+        assertImmutableRows(conn, dataTableFullName, columnEncoded);
+    }
+    
 }
  


[05/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 cc2b5b9..c8179e8 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -35,6 +35,8 @@ 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;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
@@ -42,9 +44,11 @@ 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.ENCODING_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_TYPE;
@@ -85,9 +89,14 @@ 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.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
+import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 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;
@@ -187,8 +196,12 @@ 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.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme.QualifierOutOfRangeException;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
 import org.apache.phoenix.schema.types.PDataType;
@@ -200,6 +213,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;
@@ -215,6 +229,7 @@ import org.apache.tephra.TxConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
@@ -259,8 +274,10 @@ public class MetaDataClient {
                     IS_NAMESPACE_MAPPED + "," +
                     AUTO_PARTITION_SEQ +  "," +
                     APPEND_ONLY_SCHEMA + "," +
-                    GUIDE_POSTS_WIDTH + 
-                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+                    GUIDE_POSTS_WIDTH + "," +
+                    IMMUTABLE_STORAGE_SCHEME + "," +
+                    ENCODING_SCHEME + 
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
 
     private static final String CREATE_SCHEMA = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE
             + "\"( " + TABLE_SCHEM + "," + TABLE_NAME + ") VALUES (?,?)";
@@ -275,6 +292,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 + "," +
@@ -284,14 +302,24 @@ public class MetaDataClient {
                     LINK_TYPE + "," +
                     PARENT_TENANT_ID + " " + PVarchar.INSTANCE.getSqlTypeName() + // Dynamic column for now to prevent schema change
                     ") VALUES (?, ?, ?, ?, ?, ?)";
-    private static final String INCREMENT_SEQ_NUM =
+    
+    public static final String UPDATE_ENCODED_COLUMN_COUNTER = 
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+            TENANT_ID + ", " + 
+            TABLE_SCHEM + "," +
+            TABLE_NAME + "," +
+            COLUMN_FAMILY + "," +
+            COLUMN_QUALIFIER_COUNTER + 
+            ") VALUES (?, ?, ?, ?, ?)";
+
+    public static final String INCREMENT_SEQ_NUM =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
                     TABLE_NAME + "," +
                     TABLE_SEQ_NUM  +
                     ") VALUES (?, ?, ?, ?)";
-    private static final String MUTATE_TABLE =
+    public static final String MUTATE_TABLE =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
                     TENANT_ID + "," +
                     TABLE_SCHEM + "," +
@@ -348,8 +376,10 @@ public class MetaDataClient {
                     PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
                     KEY_SEQ + "," +
                     COLUMN_DEF + "," +
+                    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 + "," +
@@ -369,8 +399,38 @@ 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 + "," +
+                    COLUMN_QUALIFIER +
+                    ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+
+    /*
+     * Custom sql to add a column to SYSTEM.CATALOG table during upgrade.
+     * We can't use the regular INSERT_COLUMN_ALTER_TABLE sql because the COLUMN_QUALIFIER column
+     * was added in 4.10. And so if upgrading from let's say 4.7, we won't be able to
+     * find the COLUMN_QUALIFIER column which the INSERT_COLUMN_ALTER_TABLE sql expects.
+     */
+    private static final String ALTER_SYSCATALOG_TABLE_UPGRADE =
+            "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                    TENANT_ID + "," +
+                    TABLE_SCHEM + "," +
+                    TABLE_NAME + "," +
+                    COLUMN_NAME + "," +
+                    COLUMN_FAMILY + "," +
+                    DATA_TYPE + "," +
+                    NULLABLE + "," +
+                    COLUMN_SIZE + "," +
+                    DECIMAL_DIGITS + "," +
+                    ORDINAL_POSITION + "," +
+                    SORT_ORDER + "," +
+                    DATA_TABLE_NAME + "," + // write this both in the column and table rows for access by metadata APIs
+                    ARRAY_SIZE + "," +
+                    VIEW_CONSTANT + "," +
+                    IS_VIEW_REFERENCED + "," +
+                    PK_NAME + "," +  // write this both in the column and table rows for access by metadata APIs
+                    KEY_SEQ + "," +
                     COLUMN_DEF +
                     ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+
     private static final String UPDATE_COLUMN_POSITION =
             "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\" ( " +
                     TENANT_ID + "," +
@@ -715,22 +775,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.getColumnForColumnName(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()) {
@@ -740,17 +800,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.getColumnForColumnName(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.getColumnForColumnName(colName);
                             }
                         } catch (ColumnNotFoundException e2) { // Ignore this index and continue with others
                             containsAllReqdCols = false;
@@ -818,7 +878,14 @@ public class MetaDataClient {
             colUpsert.setString(18, column.getExpressionStr());
         }
         if (colUpsert.getParameterMetaData().getParameterCount() > 18) {
-            colUpsert.setBoolean(19, column.isRowTimestamp());
+            if (column.getColumnQualifierBytes() == null) {
+                colUpsert.setNull(19, Types.VARBINARY);
+            } else {
+                colUpsert.setBytes(19, column.getColumnQualifierBytes());
+            }
+        }
+        if (colUpsert.getParameterMetaData().getParameterCount() > 19) {
+            colUpsert.setBoolean(20, column.isRowTimestamp());
         }
         colUpsert.execute();
     }
@@ -837,7 +904,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, byte[] columnQualifierBytes) throws SQLException {
         try {
             ColumnName columnDefName = def.getColumnDefName();
             SortOrder sortOrder = def.getSortOrder();
@@ -885,15 +952,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, columnQualifierBytes);
             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());
@@ -956,7 +1022,7 @@ public class MetaDataClient {
         }
         table = createTableInternal(statement, splits, parent, viewStatement, viewType, viewColumnConstants, isViewColumnReferenced, false, null, null, tableProps, commonFamilyProps);
 
-        if (table == null || table.getType() == PTableType.VIEW || table.isTransactional()) {
+        if (table == null || table.getType() == PTableType.VIEW /*|| table.isTransactional()*/) {
             return new MutationState(0,connection);
         }
         // Hack to get around the case when an SCN is specified on the connection.
@@ -1675,6 +1741,8 @@ public class MetaDataClient {
                     ? SchemaUtil.isNamespaceMappingEnabled(tableType, connection.getQueryServices().getProps())
                     : parent.isNamespaceMapped();
             boolean isLocalIndex = indexType == IndexType.LOCAL;
+            QualifierEncodingScheme encodingScheme = NON_ENCODED_QUALIFIERS;
+            ImmutableStorageScheme immutableStorageScheme = ONE_CELL_PER_COLUMN;
             if (parent != null && tableType == PTableType.INDEX) {
                 timestamp = TransactionUtil.getTableTimestamp(connection, transactional);
                 storeNulls = parent.getStoreNulls();
@@ -1797,7 +1865,6 @@ public class MetaDataClient {
             }
             String autoPartitionSeq = (String) TableProperty.AUTO_PARTITION_SEQ.getValue(tableProps);
             Long guidePostsWidth = (Long) TableProperty.GUIDE_POSTS_WIDTH.getValue(tableProps);
-
             Boolean storeNullsProp = (Boolean) TableProperty.STORE_NULLS.getValue(tableProps);
             if (storeNullsProp == null) {
                 if (parent == null) {
@@ -1819,7 +1886,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;
                 }
@@ -1834,7 +1901,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();
@@ -1969,7 +2036,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()) {
@@ -2013,7 +2080,105 @@ public class MetaDataClient {
             }
             int pkPositionOffset = pkColumns.size();
             int position = positionOffset;
+            EncodedCQCounter cqCounter = NULL_COUNTER;
+            PTable viewPhysicalTable = null;
+            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) {
+                    /*
+                     * 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());
+                    immutableStorageScheme = viewPhysicalTable.getImmutableStorageScheme();
+                    encodingScheme = viewPhysicalTable.getEncodingScheme();
+					if (EncodedColumnsUtil.usesEncodedColumnNames(viewPhysicalTable)) {
+                        cqCounter  = viewPhysicalTable.getEncodedCQCounter();
+                    }
+                }
+            }
+            // System tables have hard-coded column qualifiers. So we can't use column encoding for them.
+            else if (!SchemaUtil.isSystemTable(Bytes.toBytes(SchemaUtil.getTableName(schemaName, tableName)))) {
+                /*
+                 * 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 were used. 
+                 * In this case we pursue ahead with non-encoded column qualifier scheme. If the phoenix metadata for this 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 metadata already doesn't exist then the non-encoded column qualifier scheme works
+                 * because we cannot control the column qualifiers that were used when populating the hbase table.
+                 */
+                
+                byte[] tableNameBytes = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
+                boolean tableExists = true;
+                try {
+                    HTableDescriptor tableDescriptor = connection.getQueryServices().getTableDescriptor(tableNameBytes);
+                    if (tableDescriptor == null) { // for connectionless
+                        tableExists = false;
+                    }
+                } catch (org.apache.phoenix.schema.TableNotFoundException e) {
+                    tableExists = false;
+                }
+                if (tableExists) {
+                    encodingScheme = NON_ENCODED_QUALIFIERS;
+                    immutableStorageScheme = ONE_CELL_PER_COLUMN;
+                } else if (parent != null) {
+                    encodingScheme = parent.getEncodingScheme();
+                    immutableStorageScheme = parent.getImmutableStorageScheme();
+                } else {
+                	Byte encodingSchemeSerializedByte = (Byte) TableProperty.COLUMN_ENCODED_BYTES.getValue(tableProps);
+                    if (encodingSchemeSerializedByte == null) {
+                    	encodingSchemeSerializedByte = (byte)connection.getQueryServices().getProps().getInt(QueryServices.DEFAULT_COLUMN_ENCODED_BYTES_ATRRIB, QueryServicesOptions.DEFAULT_COLUMN_ENCODED_BYTES);
+                    } 
+                    encodingScheme =  QualifierEncodingScheme.fromSerializedValue(encodingSchemeSerializedByte);
+                    if (isImmutableRows) {
+                        immutableStorageScheme =
+                                (ImmutableStorageScheme) TableProperty.IMMUTABLE_STORAGE_SCHEME
+                                        .getValue(tableProps);
+                        if (immutableStorageScheme == null) {
+                            if (multiTenant) {
+                                immutableStorageScheme =
+                                        ImmutableStorageScheme
+                                                .valueOf(connection
+                                                        .getQueryServices()
+                                                        .getProps()
+                                                        .get(
+                                                            QueryServices.DEFAULT_IMMUTABLE_STORAGE_SCHEME_ATTRIB,
+                                                            QueryServicesOptions.DEFAULT_MULTITENANT_IMMUTABLE_STORAGE_SCHEME));
+                            } else {
+                                immutableStorageScheme =
+                                        ImmutableStorageScheme
+                                                .valueOf(connection
+                                                        .getQueryServices()
+                                                        .getProps()
+                                                        .get(
+                                                            QueryServices.DEFAULT_IMMUTABLE_STORAGE_SCHEME_ATTRIB,
+                                                            QueryServicesOptions.DEFAULT_IMMUTABLE_STORAGE_SCHEME));
+                            }
+                        }
+                        if (immutableStorageScheme != ONE_CELL_PER_COLUMN
+                                && encodingScheme == NON_ENCODED_QUALIFIERS) {
+                            throw new SQLExceptionInfo.Builder(
+                                    SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES)
+                                    .setSchemaName(schemaName).setTableName(tableName).build()
+                                    .buildException();
+                        }
+                    } 
+                }
+                cqCounter = encodingScheme != NON_ENCODED_QUALIFIERS ? 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...
@@ -2032,7 +2197,33 @@ public class MetaDataClient {
                                 .setColumnName(colDef.getColumnDefName().getColumnName()).build().buildException();
                     }
                 }
-                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false);
+                ColumnName columnDefName = colDef.getColumnDefName();
+                String colDefFamily = columnDefName.getFamilyName();
+                boolean isPkColumn = isPkColumn(pkConstraint, colDef, columnDefName);
+                String cqCounterFamily = null;
+                if (!isPkColumn) {
+                    if (immutableStorageScheme == SINGLE_CELL_ARRAY_WITH_OFFSETS && encodingScheme != NON_ENCODED_QUALIFIERS) {
+                        // For this scheme we track column qualifier counters at the column family level.
+                        cqCounterFamily = colDefFamily != null ? colDefFamily : (defaultFamilyName != null ? defaultFamilyName : DEFAULT_COLUMN_FAMILY);
+                    } else {
+                        // For other schemes, column qualifier counters are tracked using the default column family.
+                        cqCounterFamily = defaultFamilyName != null ? defaultFamilyName : DEFAULT_COLUMN_FAMILY;
+                    }
+                }
+                Integer encodedCQ =  isPkColumn ? null : cqCounter.getNextQualifier(cqCounterFamily);
+                byte[] columnQualifierBytes = null;
+                try {
+                    columnQualifierBytes = EncodedColumnsUtil.getColumnQualifierBytes(columnDefName.getColumnName(), encodedCQ, encodingScheme, isPkColumn);
+                }
+                catch (QualifierOutOfRangeException e) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.MAX_COLUMNS_EXCEEDED)
+                    .setSchemaName(schemaName)
+                    .setTableName(tableName).build().buildException();
+                }
+                PColumn column = newColumn(position++, colDef, pkConstraint, defaultFamilyName, false, columnQualifierBytes);
+                if (cqCounter.increment(cqCounterFamily)) {
+                    changedCqCounters.put(cqCounterFamily, cqCounter.getNextQualifier(cqCounterFamily));
+                }
                 if (SchemaUtil.isPKColumn(column)) {
                     // TODO: remove this constraint?
                     if (pkColumnsIterator.hasNext() && !column.getName().getString().equals(pkColumnsIterator.next().getFirst().getColumnName())) {
@@ -2067,6 +2258,7 @@ public class MetaDataClient {
                         column.getFamilyName());
                 }
             }
+            
             // We need a PK definition for a TABLE or mapped VIEW
             if (!isPK && pkColumnsNames.isEmpty() && tableType != PTableType.VIEW && viewType != ViewType.MAPPED) {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.PRIMARY_KEY_MISSING)
@@ -2148,15 +2340,52 @@ public class MetaDataClient {
             if (SchemaUtil.isMetaTable(schemaName,tableName)) {
                 // TODO: what about stats for system catalog?
                 PName newSchemaName = PNameFactory.newName(schemaName);
+                // Column names and qualifiers and hardcoded for system tables.
                 PTable table = PTableImpl.makePTable(tenantId,newSchemaName, PNameFactory.newName(tableName), tableType,
                         null, MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM,
                         PNameFactory.newName(QueryConstants.SYSTEM_TABLE_PK_NAME), null, columns.values(), null, null,
                         Collections.<PTable>emptyList(), isImmutableRows,
                         Collections.<PName>emptyList(), defaultFamilyName == null ? null :
                                 PNameFactory.newName(defaultFamilyName), null,
-                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema);
+                        Boolean.TRUE.equals(disableWAL), false, false, null, null, indexType, true, false, 0, 0L, isNamespaceMapped, autoPartitionSeq, isAppendOnlySchema, ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
                 connection.addTable(table, MetaDataProtocol.MIN_TABLE_TIMESTAMP);
             }
+            
+            // Update column qualifier counters
+            if (EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme)) {
+                // 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()) {
+                    PreparedStatement incrementStatement = connection.prepareStatement(INCREMENT_SEQ_NUM);
+                    incrementStatement.setString(1, null);
+                    incrementStatement.setString(2, viewPhysicalTable.getSchemaName().getString());
+                    incrementStatement.setString(3, viewPhysicalTable.getTableName().getString());
+                    incrementStatement.setLong(4, viewPhysicalTable.getSequenceNumber() + 1);
+                    incrementStatement.execute();
+                }
+                if (connection.getMutationState().toMutations(timestamp).hasNext()) {
+                    tableMetaData.addAll(connection.getMutationState().toMutations(timestamp).next().getSecond());
+                    connection.rollback();
+                }
+            }
 
             short nextKeySeq = 0;
 
@@ -2182,14 +2411,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);
@@ -2272,6 +2501,8 @@ public class MetaDataClient {
             } else {
                 tableUpsert.setLong(25, guidePostsWidth);
             }
+            tableUpsert.setByte(26, immutableStorageScheme.getSerializedMetadataValue());
+            tableUpsert.setByte(27, encodingScheme.getSerializedMetadataValue());
             tableUpsert.execute();
 
             if (asyncCreatedDate != null) {
@@ -2292,7 +2523,7 @@ public class MetaDataClient {
              * 3) parent table header row
              */
             Collections.reverse(tableMetaData);
-
+            
 			if (indexType != IndexType.LOCAL) {
                 splits = SchemaUtil.processSplits(splits, pkColumns, saltBucketNum, connection.getQueryServices().getProps().getBoolean(
                         QueryServices.FORCE_ROW_KEY_ORDER_ATTRIB, QueryServicesOptions.DEFAULT_FORCE_ROW_KEY_ORDER));
@@ -2365,12 +2596,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, immutableStorageScheme, encodingScheme, cqCounterToBe);
                 result = new MetaDataMutationResult(code, result.getMutationTime(), table, true);
                 addTableToCache(result);
                 return table;
@@ -2380,6 +2617,10 @@ public class MetaDataClient {
         }
     }
 
+    private static boolean isPkColumn(PrimaryKeyConstraint pkConstraint, ColumnDef colDef, ColumnName columnDefName) {
+        return colDef.isPK() || (pkConstraint != null && pkConstraint.getColumnWithSortOrder(columnDefName) != null);
+    }
+    
     /**
      * A table can be a parent table to tenant-specific tables if all of the following conditions are true:
      * <p>
@@ -2550,7 +2791,7 @@ public class MetaDataClient {
                                 PTable viewIndexTable = new PTableImpl(null,
                                         SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
                                         SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
-                                        table.getColumnFamilies(),table.isNamespaceMapped());
+                                        table.getColumnFamilies(),table.isNamespaceMapped(), table.getImmutableStorageScheme(), table.getEncodingScheme());
                                 tableRefs.add(new TableRef(null, viewIndexTable, ts, false));
                             }
                         }
@@ -2671,12 +2912,12 @@ public class MetaDataClient {
     }
 
     private  long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta, Boolean isTransactional, Long updateCacheFrequency) throws SQLException {
-        return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, updateCacheFrequency, null, null, null, null, -1L, null);
+        return incrementTableSeqNum(table, expectedType, columnCountDelta, isTransactional, updateCacheFrequency, null, null, null, null, -1L, null, null);
     }
 
     private long incrementTableSeqNum(PTable table, PTableType expectedType, int columnCountDelta,
             Boolean isTransactional, Long updateCacheFrequency, Boolean isImmutableRows, Boolean disableWAL,
-            Boolean isMultiTenant, Boolean storeNulls, Long guidePostWidth, Boolean appendOnlySchema)
+            Boolean isMultiTenant, Boolean storeNulls, Long guidePostWidth, Boolean appendOnlySchema, ImmutableStorageScheme immutableStorageScheme)
             throws SQLException {
         String schemaName = table.getSchemaName().getString();
         String tableName = table.getTableName().getString();
@@ -2720,6 +2961,10 @@ public class MetaDataClient {
         if (appendOnlySchema !=null) {
             mutateBooleanProperty(tenantId, schemaName, tableName, APPEND_ONLY_SCHEMA, appendOnlySchema);
         }
+        if (immutableStorageScheme !=null) {
+            mutateStringProperty(tenantId, schemaName, tableName, IMMUTABLE_STORAGE_SCHEME, immutableStorageScheme.name());
+        }
+        
         return seqNum;
     }
 
@@ -2760,6 +3005,23 @@ public class MetaDataClient {
             tableBoolUpsert.execute();
         }
     }
+    
+    private void mutateStringProperty(String tenantId, String schemaName, String tableName,
+            String propertyName, String propertyValue) throws SQLException {
+        String updatePropertySql = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                propertyName +
+                ") VALUES (?, ?, ?, ?)";
+        try (PreparedStatement tableBoolUpsert = connection.prepareStatement(updatePropertySql)) {
+            tableBoolUpsert.setString(1, tenantId);
+            tableBoolUpsert.setString(2, schemaName);
+            tableBoolUpsert.setString(3, tableName);
+            tableBoolUpsert.setString(4, propertyValue);
+            tableBoolUpsert.execute();
+        }
+    }
 
     public MutationState addColumn(AddColumnStatement statement) throws SQLException {
         PTable table = FromCompiler.getResolver(statement, connection).getTables().get(0).getTable();
@@ -2777,7 +3039,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;
@@ -2786,6 +3047,7 @@ public class MetaDataClient {
             Long updateCacheFrequencyProp = null;
             Boolean appendOnlySchemaProp = null;
             Long guidePostWidth = -1L;
+            ImmutableStorageScheme immutableStorageSchemeProp = null;
 
             Map<String, List<Pair<String, Object>>> properties = new HashMap<>(stmtProperties.size());
             List<ColumnDef> columnDefs = null;
@@ -2798,7 +3060,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);
                             }
@@ -2809,7 +3071,7 @@ public class MetaDataClient {
                     }
                     else {
                         try {
-                            table.getColumn(columnName);
+                            table.getColumnForColumnName(columnName);
                             if (!ifNotExists) {
                                 throw new ColumnAlreadyExistsException(schemaName, tableName, columnName);
                             }
@@ -2848,6 +3110,8 @@ public class MetaDataClient {
                             guidePostWidth = (Long)value;
                         } else if (propName.equals(APPEND_ONLY_SCHEMA)) {
                             appendOnlySchemaProp = (Boolean) value;
+                        } else if (propName.equalsIgnoreCase(IMMUTABLE_STORAGE_SCHEME)) {
+                            immutableStorageSchemeProp = (ImmutableStorageScheme)value;
                         }
                     }
                     // if removeTableProps is true only add the property if it is not a HTable or Phoenix Table property
@@ -2864,7 +3128,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()) {
@@ -2889,6 +3154,10 @@ public class MetaDataClient {
                 Boolean isImmutableRows = null;
                 if (isImmutableRowsProp != null) {
                     if (isImmutableRowsProp.booleanValue() != table.isImmutableRows()) {
+                    	if (table.getImmutableStorageScheme() != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
+                    		throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY)
+                    		.setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                    	}
                         isImmutableRows = isImmutableRowsProp;
                         changingPhoenixTableProperty = true;
                     }
@@ -2921,6 +3190,18 @@ public class MetaDataClient {
                         changingPhoenixTableProperty = true;
                     }
                 }
+                ImmutableStorageScheme immutableStorageScheme = null;
+                if (immutableStorageSchemeProp!=null) {
+                    if (table.getImmutableStorageScheme() == ONE_CELL_PER_COLUMN || 
+                            immutableStorageSchemeProp == ONE_CELL_PER_COLUMN) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE)
+                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                    }
+                    else if (immutableStorageSchemeProp != table.getImmutableStorageScheme()) {
+                        immutableStorageScheme = immutableStorageSchemeProp;
+                        changingPhoenixTableProperty = true;
+                    }
+                }
             
                 if (guidePostWidth == null || guidePostWidth >= 0) {
                     changingPhoenixTableProperty = true;
@@ -2965,12 +3246,19 @@ public class MetaDataClient {
                 Long timeStamp = TransactionUtil.getTableTimestamp(connection, table.isTransactional() || nonTxToTx);
 
                 int numPkColumnsAdded = 0;
-                List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnDefs.size());
+                List<PColumn> columns = Lists.newArrayListWithExpectedSize(numCols);
                 Set<String> colFamiliesForPColumnsToBeAdded = new LinkedHashSet<>();
                 Set<String> families = new LinkedHashSet<>();
-                if (columnDefs.size() > 0 ) {
+                PTable tableForCQCounters = tableType == PTableType.VIEW ? PhoenixRuntime.getTable(connection, table.getPhysicalName().getString()) : table;;
+                EncodedCQCounter cqCounterToUse = tableForCQCounters.getEncodedCQCounter();
+                Map<String, Integer> changedCqCounters = new HashMap<>(numCols);
+                if (numCols > 0 ) {
                     StatementContext context = new StatementContext(new PhoenixStatement(connection), resolver);
-                    try (PreparedStatement colUpsert = connection.prepareStatement(INSERT_COLUMN_ALTER_TABLE)) {
+                    String addColumnSqlToUse = connection.isRunningUpgrade()
+                            && tableName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE)
+                            && schemaName.equals(PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA) ? ALTER_SYSCATALOG_TABLE_UPGRADE
+                            : INSERT_COLUMN_ALTER_TABLE;
+                    try (PreparedStatement colUpsert = connection.prepareStatement(addColumnSqlToUse)) {
                         short nextKeySeq = SchemaUtil.getMaxKeySeq(table);
                         for( ColumnDef colDef : columnDefs) {
                             if (colDef != null && !colDef.isNull()) {
@@ -2992,11 +3280,41 @@ public class MetaDataClient {
                             if (!colDef.validateDefault(context, null)) {
                                 colDef = new ColumnDef(colDef, null); // Remove DEFAULT as it's not necessary
                             }
-                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true);
+                            Integer encodedCQ = null;
+                            if (!colDef.isPK()) {
+                                String colDefFamily = colDef.getColumnDefName().getFamilyName();
+                                String familyName = null;
+                                ImmutableStorageScheme storageScheme = table.getImmutableStorageScheme();
+                                String defaultColumnFamily = tableForCQCounters.getDefaultFamilyName() != null && !Strings.isNullOrEmpty(tableForCQCounters.getDefaultFamilyName().getString()) ? 
+                                        tableForCQCounters.getDefaultFamilyName().getString() : DEFAULT_COLUMN_FAMILY;
+                                    if (table.getType() == PTableType.INDEX && table.getIndexType() == IndexType.LOCAL) {
+                                        defaultColumnFamily = QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX + defaultColumnFamily;
+                                    }
+                                if (storageScheme == SINGLE_CELL_ARRAY_WITH_OFFSETS) {
+                                    familyName = colDefFamily != null ? colDefFamily : defaultColumnFamily;
+                                } else {
+                                    familyName = defaultColumnFamily;
+                                }
+                                encodedCQ = cqCounterToUse.getNextQualifier(familyName);
+                                if (cqCounterToUse.increment(familyName)) {
+                                    changedCqCounters.put(familyName,
+                                        cqCounterToUse.getNextQualifier(familyName));
+                                }
+                            }
+                            byte[] columnQualifierBytes = null;
+                            try {
+                                columnQualifierBytes = EncodedColumnsUtil.getColumnQualifierBytes(colDef.getColumnDefName().getColumnName(), encodedCQ, table, colDef.isPK());
+                            }
+                            catch (QualifierOutOfRangeException e) {
+                                throw new SQLExceptionInfo.Builder(SQLExceptionCode.MAX_COLUMNS_EXCEEDED)
+                                .setSchemaName(schemaName)
+                                .setTableName(tableName).build().buildException();
+                            }
+                            PColumn column = newColumn(position++, colDef, PrimaryKeyConstraint.EMPTY, table.getDefaultFamilyName() == null ? null : table.getDefaultFamilyName().getString(), true, columnQualifierBytes);
                             columns.add(column);
                             String pkName = null;
                             Short keySeq = null;
-
+                            
                             // TODO: support setting properties on other families?
                             if (column.getFamilyName() == null) {
                                 ++numPkColumnsAdded;
@@ -3008,13 +3326,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));
                                 }
@@ -3023,14 +3341,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);
                                     }
                                 }
@@ -3066,10 +3384,10 @@ public class MetaDataClient {
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
-                long seqNum = table.getSequenceNumber();
+                
                 if (changingPhoenixTableProperty || columnDefs.size() > 0) {
-                    seqNum = incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows,
-                            disableWAL, multiTenant, storeNulls, guidePostWidth, appendOnlySchema);
+                    incrementTableSeqNum(table, tableType, columnDefs.size(), isTransactional, updateCacheFrequency, isImmutableRows,
+                            disableWAL, multiTenant, storeNulls, guidePostWidth, appendOnlySchema, immutableStorageScheme);
                     tableMetaData.addAll(connection.getMutationState().toMutations(timeStamp).next().getSecond());
                     connection.rollback();
                 }
@@ -3078,6 +3396,33 @@ public class MetaDataClient {
                 Collections.reverse(tableMetaData);
                 // Add column metadata afterwards, maintaining the order so columns have more predictable ordinal position
                 tableMetaData.addAll(columnMetaData);
+                boolean sharedIndex = tableType == PTableType.INDEX && (table.getIndexType() == IndexType.LOCAL || table.getViewIndexId() != null);
+                String tenantIdToUse = connection.getTenantId() != null && sharedIndex ? connection.getTenantId().getString() : null;
+                if (!changedCqCounters.isEmpty()) {
+                    PreparedStatement linkStatement;
+                        linkStatement = connection.prepareStatement(UPDATE_ENCODED_COLUMN_COUNTER);
+                        for (Entry<String, Integer> entry : changedCqCounters.entrySet()) {    
+                            linkStatement.setString(1, tenantIdToUse);
+                            linkStatement.setString(2, tableForCQCounters.getSchemaName().getString());
+                            linkStatement.setString(3, tableForCQCounters.getTableName().getString());
+                            linkStatement.setString(4, entry.getKey());
+                            linkStatement.setInt(5, entry.getValue());
+                            linkStatement.execute();
+                        }
+
+                    // When a view adds its own columns, then we need to increase the sequence number of the base table
+                    // too since we want clients to get the latest PTable of the base table.
+                    if (tableType == VIEW) {
+                        PreparedStatement incrementStatement = connection.prepareStatement(INCREMENT_SEQ_NUM);
+                        incrementStatement.setString(1, null);
+                        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;
 
@@ -3107,7 +3452,6 @@ public class MetaDataClient {
                         }
                         return new MutationState(0,connection);
                     }
-
                     // Only update client side cache if we aren't adding a PK column to a table with indexes or
                     // transitioning a table from non transactional to transactional.
                     // We could update the cache manually then too, it'd just be a pain.
@@ -3115,6 +3459,7 @@ public class MetaDataClient {
                     long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result);
                     if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && !nonTxToTx)) {
                         connection.addTable(result.getTable(), resolvedTimeStamp);
+                        table = result.getTable();
                     } else if (updateCacheFrequency != null) {
                         // Force removal from cache as the update cache frequency has changed
                         // Note that clients outside this JVM won't be affected.
@@ -3139,7 +3484,7 @@ public class MetaDataClient {
                             PTable viewIndexTable = new PTableImpl(null,
                                     SchemaUtil.getSchemaNameFromFullName(viewIndexPhysicalName),
                                     SchemaUtil.getTableNameFromFullName(viewIndexPhysicalName), ts,
-                                    table.getColumnFamilies(), table.isNamespaceMapped());
+                                    table.getColumnFamilies(), table.isNamespaceMapped(), table.getImmutableStorageScheme(), table.getEncodingScheme());
                             List<TableRef> tableRefs = Collections.singletonList(new TableRef(null, viewIndexTable, ts, false));
                             MutationPlan plan = new PostDDLCompiler(connection).compile(tableRefs, null, null,
                                     Collections.<PColumn> emptyList(), ts);
@@ -3291,23 +3636,24 @@ 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<ColumnReference> coveredCols = indexMaintainer.getCoveredColumns();
                     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());
+                        ColumnReference colDropRef = new ColumnReference(columnToDrop.getFamilyName() == null ? null
+                                : columnToDrop.getFamilyName().getBytes(), columnToDrop.getColumnQualifierBytes());
+                        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 (coveredCols.contains(colDropRef)) {
                             String indexColumnName = IndexUtil.getIndexColumnName(columnToDrop);
-                            PColumn indexColumn = index.getColumn(indexColumnName);
+                            PColumn indexColumn = index.getColumnForColumnName(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()));
@@ -3397,13 +3743,13 @@ 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()) {
                                 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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
                                 TableRef indexTableRef = new TableRef(viewIndexTable);
                                 PName indexTableTenantId = sharedTableState.getTenantId();
                                 if (indexTableTenantId==null) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..9e26227 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumn.java
@@ -27,7 +27,7 @@ package org.apache.phoenix.schema;
 public interface PColumn extends PDatum {
 
     /**
-     * @return the name of the column qualifier
+     * @return the name of the column
      */
     PName getName();
 
@@ -60,4 +60,6 @@ public interface PColumn extends PDatum {
     boolean isRowTimestamp();
     
     boolean isDynamic();
+    
+    byte[] getColumnQualifierBytes();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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..453e33b 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
@@ -17,10 +17,14 @@
  */
 package org.apache.phoenix.schema;
 
+import static org.apache.phoenix.util.EncodedColumnsUtil.usesEncodedColumnNames;
+
 import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
 
 import com.google.common.base.Preconditions;
@@ -31,8 +35,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> columnsByQualifiers;
     private final int estimatedSize;
 
     @Override
@@ -47,15 +52,23 @@ public class PColumnFamilyImpl implements PColumnFamily {
                 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> columnsByQualifiersBuilder = 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);
+            // In certain cases like JOIN, PK columns are assigned a column family. So they
+            // are not evaluated as a PK column. However, their column qualifier bytes are
+            // still null.
+            if (!SchemaUtil.isPKColumn(column) && column.getColumnQualifierBytes() != null) {
+                columnsByQualifiersBuilder.put(column.getColumnQualifierBytes(), column);
+            }
         }
-        this.columnByBytes = columnByBytesBuilder.build();
-        this.columnByString = columnByStringBuilder.build();
+        this.columnNamesByBytes = columnNamesByBytesBuilder.build();
+        this.columnNamesByStrings = columnNamesByStringBuilder.build();
+        this.columnsByQualifiers =  columnsByQualifiersBuilder.build();
         this.estimatedSize = (int)estimatedSize;
     }
     
@@ -70,20 +83,28 @@ 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;
     }
+    
+    
+    //TODO: samarth think about backward compatibility here
+    @Override
+    public PColumn getPColumnForColumnQualifier(byte[] cq) throws ColumnNotFoundException {
+        Preconditions.checkNotNull(cq);
+        return columnsByQualifiers.get(cq);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PColumnImpl.java
index ca827d8..78baa4c 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
@@ -21,6 +21,7 @@ import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
 
 import com.google.common.base.Preconditions;
@@ -40,6 +41,7 @@ public class PColumnImpl implements PColumn {
     private String expressionStr;
     private boolean isRowTimestamp;
     private boolean isDynamic;
+    private byte[] columnQualifierBytes;
     
     public PColumnImpl() {
     }
@@ -51,13 +53,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, byte[] columnQualifierBytes) {
+        init(name, familyName, dataType, maxLength, scale, nullable, position, sortOrder, arrSize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes);
     }
 
     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.getColumnQualifierBytes());
     }
 
     private void init(PName name,
@@ -69,7 +71,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, byte[] columnQualifierBytes) {
     	Preconditions.checkNotNull(sortOrder);
         this.dataType = dataType;
         if (familyName == null) {
@@ -94,6 +96,7 @@ public class PColumnImpl implements PColumn {
         this.expressionStr = expressionStr;
         this.isRowTimestamp = isRowTimestamp;
         this.isDynamic = isDynamic;
+        this.columnQualifierBytes = columnQualifierBytes;
     }
 
     @Override
@@ -205,6 +208,15 @@ public class PColumnImpl implements PColumn {
     public boolean isDynamic() {
         return isDynamic;
     }
+    
+    @Override
+    public byte[] getColumnQualifierBytes() {
+        // Needed for backward compatibility
+        if (!SchemaUtil.isPKColumn(this) && columnQualifierBytes == null) {
+            return this.name.getBytes();
+        }
+        return columnQualifierBytes;
+    }
 
     /**
      * Create a PColumn instance from PBed PColumn instance
@@ -251,8 +263,12 @@ public class PColumnImpl implements PColumn {
         if (column.hasIsDynamic()) {
         	isDynamic = column.getIsDynamic();
         }
+        byte[] columnQualifierBytes = null;
+        if (column.hasColumnQualifierBytes()) {
+            columnQualifierBytes = column.getColumnQualifierBytes().toByteArray();
+        }
         return new PColumnImpl(columnName, familyName, dataType, maxLength, scale, nullable, position, sortOrder,
-                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic);
+                arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, isDynamic, columnQualifierBytes);
     }
 
     public static PTableProtos.PColumn toProto(PColumn column) {
@@ -283,6 +299,9 @@ public class PColumnImpl implements PColumn {
             builder.setExpression(column.getExpressionStr());
         }
         builder.setIsRowTimestamp(column.isRowTimestamp());
+        if (column.getColumnQualifierBytes() != null) {
+            builder.setColumnQualifierBytes(ByteStringer.wrap(column.getColumnQualifierBytes()));
+        }
         return builder.build();
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 6ca38cc..169e78d 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
@@ -199,7 +199,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();
@@ -214,7 +214,7 @@ public class PMetaDataImpl implements PMetaData {
             // Update position of columns that follow removed column
             for (int i = position+1; i < oldColumns.size(); i++) {
                 PColumn oldColumn = oldColumns.get(i);
-                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic());
+                PColumn newColumn = new PColumnImpl(oldColumn.getName(), oldColumn.getFamilyName(), oldColumn.getDataType(), oldColumn.getMaxLength(), oldColumn.getScale(), oldColumn.isNullable(), i-1+positionOffset, oldColumn.getSortOrder(), oldColumn.getArraySize(), oldColumn.getViewConstant(), oldColumn.isViewReferenced(), oldColumn.getExpressionStr(), oldColumn.isRowTimestamp(), oldColumn.isDynamic(), oldColumn.getColumnQualifierBytes());
                 columns.add(newColumn);
             }
             


[08/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index fde403c..8595eda 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -368,6 +368,10 @@ public enum SQLExceptionCode {
     CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW(1134, "XCL34", "Altering this table property on a view is not allowed"),
     
     IMMUTABLE_TABLE_PROPERTY_INVALID(1135, "XCL35", "IMMUTABLE table property cannot be used with CREATE IMMUTABLE TABLE statement "),
+    
+    MAX_COLUMNS_EXCEEDED(1136, "XCL36", "The number of columns exceed the maximum supported by the table's qualifier encoding scheme"),
+    INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES(1137, "XCL37", "If IMMUTABLE_STORAGE_SCHEME property is not set to ONE_CELL_PER_COLUMN COLUMN_ENCODED_BYTES cannot be 0"),
+    INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE(1138, "XCL38", "IMMUTABLE_STORAGE_SCHEME property cannot be changed from/to ONE_CELL_PER_COLUMN "),
 
     /**
      * Implementation defined class. Phoenix internal error. (errorcode 20, sqlstate INT).

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 76dec2f..f6010ac 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
@@ -65,11 +65,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.ImmutableStorageScheme;
 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;
@@ -313,10 +315,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();
@@ -327,6 +325,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.
@@ -373,7 +377,7 @@ public abstract class BaseQueryPlan implements QueryPlan {
         }
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         IndexMaintainer.serialize(dataTable, ptr, indexes, context.getConnection());
-        scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD, ByteUtil.copyKeyBytesIfNecessary(ptr));
+        scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD_PROTO, ByteUtil.copyKeyBytesIfNecessary(ptr));
         if (dataTable.isTransactional()) {
             scan.setAttribute(BaseScannerRegionObserver.TX_STATE, context.getConnection().getMutationState().encodeTransaction());
         }
@@ -429,14 +433,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.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+            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 = column.getColumnQualifierBytes();
+                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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
index b1d00ab..ee81c36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/CorrelatePlan.java
@@ -159,7 +159,7 @@ public class CorrelatePlan extends DelegateQueryPlan {
                     joined = rhsBitSet == ValueBitSet.EMPTY_VALUE_BITSET ?
                             current : TupleProjector.mergeProjectedValue(
                                     convertLhs(current), joinedSchema, destBitSet,
-                                    rhsCurrent, rhsSchema, rhsBitSet, rhsFieldPosition);
+                                    rhsCurrent, rhsSchema, rhsBitSet, rhsFieldPosition, true);
                 } catch (IOException e) {
                     throw new SQLException(e);
                 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 4775d59..d32199b 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
@@ -588,7 +588,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) {
@@ -622,6 +622,7 @@ public class MutationState implements SQLCloseable {
         Iterator<Map.Entry<ImmutableBytesPtr, RowMutationState>> iterator =
                 values.entrySet().iterator();
         long timestampToUse = timestamp;
+        Map<ImmutableBytesPtr, RowMutationState> modifiedValues = Maps.newHashMap();
         while (iterator.hasNext()) {
             Map.Entry<ImmutableBytesPtr, RowMutationState> rowEntry = iterator.next();
             byte[] onDupKeyBytes = rowEntry.getValue().getOnDupKeyBytes();
@@ -631,6 +632,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 {
@@ -671,6 +676,7 @@ public class MutationState implements SQLCloseable {
             if (mutationsPertainingToIndex != null) mutationsPertainingToIndex
                     .addAll(rowMutationsPertainingToIndex);
         }
+        values.putAll(modifiedValues);
     }
     
     /**
@@ -808,7 +814,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());
                     }
                 }
             }
@@ -1228,7 +1234,7 @@ public class MutationState implements SQLCloseable {
             }
             mutation.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
             if (attribValue != null) {
-                mutation.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+                mutation.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
                 if (txState.length > 0) {
                     mutation.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
                 }
@@ -1523,8 +1529,8 @@ public class MutationState implements SQLCloseable {
         byte[] getOnDupKeyBytes() {
             return onDupKeyBytes;
         }
-        
-        Map<PColumn, byte[]> getColumnValues() {
+
+        public Map<PColumn, byte[]> getColumnValues() {
             return columnValues;
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..8913f3b 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
@@ -414,7 +414,7 @@ public class SortMergeJoinPlan implements QueryPlan {
                 return rhsBitSet == ValueBitSet.EMPTY_VALUE_BITSET ?
                         t : TupleProjector.mergeProjectedValue(
                                 t, joinedSchema, destBitSet,
-                                rhs, rhsSchema, rhsBitSet, rhsFieldPosition);
+                                rhs, rhsSchema, rhsBitSet, rhsFieldPosition, true);
             } catch (IOException e) {
                 throw new SQLException(e);
             }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
index 592b68e..2126026 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/TupleProjector.java
@@ -17,6 +17,9 @@
  */
 package org.apache.phoenix.execute;
 
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_QUALIFIER;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -51,9 +54,6 @@ import org.apache.phoenix.util.SchemaUtil;
 import com.google.common.base.Preconditions;
 
 public class TupleProjector {    
-    public static final byte[] VALUE_COLUMN_FAMILY = Bytes.toBytes("_v");
-    public static final byte[] VALUE_COLUMN_QUALIFIER = new byte[0];
-    
     private static final String SCAN_PROJECTOR = "scanProjector";
     
     private final KeyValueSchema schema;
@@ -61,6 +61,8 @@ public class TupleProjector {
     private ValueBitSet valueSet;
     private final ImmutableBytesWritable ptr = new ImmutableBytesWritable();
     
+    private static final byte[] OLD_VALUE_COLUMN_QUALIFIER = new byte[0];
+    
     public TupleProjector(RowProjector rowProjector) {
         List<? extends ColumnProjector> columnProjectors = rowProjector.getColumnProjectors();
         int count = columnProjectors.size();
@@ -165,11 +167,11 @@ public class TupleProjector {
     }
     
     public static class ProjectedValueTuple extends BaseTuple {
-        private ImmutableBytesWritable keyPtr = new ImmutableBytesWritable();
-        private long timestamp;
-        private ImmutableBytesWritable projectedValue = new ImmutableBytesWritable();
-        private int bitSetLen;
-        private KeyValue keyValue;
+        ImmutableBytesWritable keyPtr = new ImmutableBytesWritable();
+        long timestamp;
+        ImmutableBytesWritable projectedValue = new ImmutableBytesWritable();
+        int bitSetLen;
+        KeyValue keyValue;
 
         public ProjectedValueTuple(Tuple keyBase, long timestamp, byte[] projectedValue, int valueOffset, int valueLength, int bitSetLen) {
             keyBase.getKey(this.keyPtr);
@@ -241,20 +243,63 @@ public class TupleProjector {
         }
     }
     
+    public static class OldProjectedValueTuple extends ProjectedValueTuple {
+
+        public OldProjectedValueTuple(byte[] keyBuffer, int keyOffset, int keyLength, long timestamp,
+                byte[] projectedValue, int valueOffset, int valueLength, int bitSetLen) {
+            super(keyBuffer, keyOffset, keyLength, timestamp, projectedValue, valueOffset, valueLength, bitSetLen);
+        }
+
+        public OldProjectedValueTuple(Tuple keyBase, long timestamp, byte[] projectedValue, int valueOffset,
+                int valueLength, int bitSetLen) {
+            super(keyBase, timestamp, projectedValue, valueOffset, valueLength, bitSetLen);
+        }
+
+        @Override
+        public KeyValue getValue(int index) {
+            if (index != 0) { throw new IndexOutOfBoundsException(Integer.toString(index)); }
+            return getValue(VALUE_COLUMN_FAMILY, OLD_VALUE_COLUMN_QUALIFIER);
+        }
+
+        @Override
+        public KeyValue getValue(byte[] family, byte[] qualifier) {
+            if (keyValue == null) {
+                keyValue = KeyValueUtil.newKeyValue(keyPtr.get(), keyPtr.getOffset(), keyPtr.getLength(),
+                        VALUE_COLUMN_FAMILY, OLD_VALUE_COLUMN_QUALIFIER, timestamp, projectedValue.get(),
+                        projectedValue.getOffset(), projectedValue.getLength());
+            }
+            return keyValue;
+        }
+        
+    }
+    
     public ProjectedValueTuple projectResults(Tuple tuple) {
     	byte[] bytesValue = schema.toBytes(tuple, getExpressions(), valueSet, ptr);
     	Cell base = tuple.getValue(0);
         return new ProjectedValueTuple(base.getRowArray(), base.getRowOffset(), base.getRowLength(), base.getTimestamp(), bytesValue, 0, bytesValue.length, valueSet.getEstimatedLength());
     }
     
+    public ProjectedValueTuple projectResults(Tuple tuple, boolean useNewValueQualifier) {
+        byte[] bytesValue = schema.toBytes(tuple, getExpressions(), valueSet, ptr);
+        Cell base = tuple.getValue(0);
+        if (useNewValueQualifier) {
+            return new ProjectedValueTuple(base.getRowArray(), base.getRowOffset(), base.getRowLength(), base.getTimestamp(), bytesValue, 0, bytesValue.length, valueSet.getEstimatedLength());
+        } else {
+            return new OldProjectedValueTuple(base.getRowArray(), base.getRowOffset(), base.getRowLength(), base.getTimestamp(), bytesValue, 0, bytesValue.length, valueSet.getEstimatedLength());
+        }
+    }
+    
     public static void decodeProjectedValue(Tuple tuple, ImmutableBytesWritable ptr) throws IOException {
-    	boolean b = tuple.getValue(VALUE_COLUMN_FAMILY, VALUE_COLUMN_QUALIFIER, ptr);
-        if (!b)
-            throw new IOException("Trying to decode a non-projected value.");
+        boolean b = tuple.getValue(VALUE_COLUMN_FAMILY, VALUE_COLUMN_QUALIFIER, ptr);
+        if (!b) {
+            // fall back to use the old value column qualifier for backward compatibility
+            b = tuple.getValue(VALUE_COLUMN_FAMILY, OLD_VALUE_COLUMN_QUALIFIER, ptr);
+        }
+        if (!b) throw new IOException("Trying to decode a non-projected value.");
     }
     
     public static ProjectedValueTuple mergeProjectedValue(ProjectedValueTuple dest, KeyValueSchema destSchema, ValueBitSet destBitSet,
-    		Tuple src, KeyValueSchema srcSchema, ValueBitSet srcBitSet, int offset) throws IOException {
+    		Tuple src, KeyValueSchema srcSchema, ValueBitSet srcBitSet, int offset, boolean useNewValueColumnQualifier) throws IOException {
     	ImmutableBytesWritable destValue = dest.getProjectedValue();
         int origDestBitSetLen = dest.getBitSetLength();
     	destBitSet.clear();
@@ -281,7 +326,8 @@ public class TupleProjector {
     	    o = Bytes.putBytes(merged, o, srcValue.get(), srcValue.getOffset(), srcValueLen);
     	}
     	destBitSet.toBytes(merged, o);
-        return new ProjectedValueTuple(dest, dest.getTimestamp(), merged, 0, merged.length, destBitSetLen);
+        return useNewValueColumnQualifier ? new ProjectedValueTuple(dest, dest.getTimestamp(), merged, 0, merged.length, destBitSetLen) : 
+            new OldProjectedValueTuple(dest, dest.getTimestamp(), merged, 0, merged.length, destBitSetLen);
     }
 
     public KeyValueSchema getSchema() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java b/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
index bda1b96..51cb67e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/execute/UnnestArrayPlan.java
@@ -33,6 +33,7 @@ import org.apache.phoenix.iterate.ParallelScanGrouper;
 import org.apache.phoenix.iterate.ResultIterator;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PInteger;
 
@@ -138,7 +139,7 @@ public class UnnestArrayPlan extends DelegateQueryPlan {
         @Override
         public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
             ptr.set(arrayPtr.get(), arrayPtr.getOffset(), arrayPtr.getLength());
-            PArrayDataType.positionAtArrayElement(ptr, index++, getDataType(), getMaxLength());
+            PArrayDataTypeDecoder.positionAtArrayElement(ptr, index++, getDataType(), getMaxLength());
             return true;
         }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/ArrayConstructorExpression.java
index c2f4dd2..8b83bf7 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
@@ -21,6 +21,7 @@ import org.apache.hadoop.io.WritableUtils;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
@@ -31,13 +32,9 @@ import org.apache.phoenix.util.TrustedByteArrayOutputStream;
 public class ArrayConstructorExpression extends BaseCompoundExpression {
     private PDataType baseType;
     private int position = -1;
-    private int nNulls = 0;
     private Object[] elements;
     private final ImmutableBytesWritable valuePtr = new ImmutableBytesWritable();
     private int estimatedSize = 0;
-    // store the offset postion in this.  Later based on the total size move this to a byte[]
-    // and serialize into byte stream
-    private int[] offsetPos;
     private boolean rowKeyOrderOptimizable;
     
     public ArrayConstructorExpression() {
@@ -58,9 +55,6 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
         elements = new Object[getChildren().size()];
         valuePtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
         estimatedSize = PArrayDataType.estimateSize(this.children.size(), this.baseType);
-        if (!this.baseType.isFixedWidth()) {
-            offsetPos = new int[children.size()];
-        }
     }
 
     @Override
@@ -72,7 +66,6 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
     public void reset() {
         super.reset();
         position = 0;
-        nNulls = 0;
         Arrays.fill(elements, null);
         valuePtr.set(ByteUtil.EMPTY_BYTE_ARRAY);
     }
@@ -85,66 +78,24 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
         }
         TrustedByteArrayOutputStream byteStream = new TrustedByteArrayOutputStream(estimatedSize);
         DataOutputStream oStream = new DataOutputStream(byteStream);
-        try {
-            int noOfElements =  children.size();
-            nNulls = 0;
-            for (int i = position >= 0 ? position : 0; i < elements.length; i++) {
-                Expression child = children.get(i);
-                if (!child.evaluate(tuple, ptr)) {
-                    if (tuple != null && !tuple.isImmutable()) {
-                        if (position >= 0) position = i;
-                        return false;
-                    }
-                } else {
-                    // track the offset position here from the size of the byteStream
-                    if (!baseType.isFixedWidth()) {
-                        // Any variable length array would follow the below order
-                        // Every element would be seperated by a seperator byte '0'
-                        // Null elements are counted and once a first non null element appears we
-                        // write the count of the nulls prefixed with a seperator byte
-                        // Trailing nulls are not taken into account
-                        // The last non null element is followed by two seperator bytes
-                        // For eg
-                        // a, b, null, null, c, null would be 
-                        // 65 0 66 0 0 2 67 0 0 0
-                        // a null null null b c null d would be
-                        // 65 0 0 3 66 0 67 0 0 1 68 0 0 0
-                        if (ptr.getLength() == 0) {
-                            offsetPos[i] = byteStream.size();
-                            nNulls++;
-                        } else {
-                            PArrayDataType.serializeNulls(oStream, nNulls);
-                            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());
-                    }
+        PArrayDataTypeEncoder builder =
+                new PArrayDataTypeEncoder(byteStream, oStream, children.size(), baseType, getSortOrder(), rowKeyOrderOptimizable, PArrayDataType.SORTABLE_SERIALIZATION_VERSION);
+        for (int i = position >= 0 ? position : 0; i < elements.length; i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                if (tuple != null && !tuple.isImmutable()) {
+                    if (position >= 0) position = i;
+                    return false;
                 }
-            }
-            if (position >= 0) position = elements.length;
-            if (!baseType.isFixedWidth()) {
-                // Double seperator byte to show end of the non null array
-                PArrayDataType.writeEndSeperatorForVarLengthArray(oStream, getSortOrder(), rowKeyOrderOptimizable);
-                noOfElements = PArrayDataType.serailizeOffsetArrayIntoStream(oStream, byteStream, noOfElements,
-                        offsetPos[offsetPos.length - 1], offsetPos);
-                PArrayDataType.serializeHeaderInfoIntoStream(oStream, noOfElements);
-            }
-            ptr.set(byteStream.getBuffer(), 0, byteStream.size());
-            valuePtr.set(ptr.get(), ptr.getOffset(), ptr.getLength());
-            return true;
-        } catch (IOException e) {
-            throw new RuntimeException("Exception while serializing the byte array");
-        } finally {
-            try {
-                byteStream.close();
-                oStream.close();
-            } catch (IOException e) {
-                // Should not happen
+            } else {
+                builder.appendValue(ptr.get(), ptr.getOffset(), ptr.getLength());
             }
         }
+        if (position >= 0) position = elements.length;
+        byte[] bytes = builder.encode();
+        ptr.set(bytes, 0, bytes.length);
+        valuePtr.set(ptr.get(), ptr.getOffset(), ptr.getLength());
+        return true;
     }
 
 
@@ -196,4 +147,5 @@ public class ArrayConstructorExpression extends BaseCompoundExpression {
         buf.append(children.get(children.size()-1) + "]");
         return buf.toString();
     }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 aa08adf..f5f4a29 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
@@ -287,9 +287,8 @@ public enum ExpressionType {
     CeilYearExpression(CeilYearExpression.class),
     DayOfWeekFunction(DayOfWeekFunction.class),
     DayOfYearFunction(DayOfYearFunction.class),
-    DefaultValueExpression(DefaultValueExpression.class);
-
-
+    DefaultValueExpression(DefaultValueExpression.class),
+    ArrayColumnExpression(SingleCellColumnExpression.class);
 
     ExpressionType(Class<? extends Expression> clazz) {
         this.clazz = clazz;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..f8432c5 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
@@ -41,33 +41,38 @@ 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.
 
     public KeyValueColumnExpression() {
     }
-
+    
     public KeyValueColumnExpression(PColumn column) {
-        this(column, null);
-    }
-
-    public KeyValueColumnExpression(PDatum column, byte[] cf, byte[] cq) {
         super(column);
-        this.cf = cf;
-        this.cq = cq;
+        this.cf = column.getFamilyName().getBytes();
+        // for backward compatibility since older tables won't have columnQualifierBytes in their metadata
+        this.cq = column.getColumnQualifierBytes() != null ? column.getColumnQualifierBytes() : column.getName().getBytes();
+        this.displayName = column.getName().getString();
     }
-
+    
     public KeyValueColumnExpression(PColumn column, String displayName) {
         super(column);
         this.cf = column.getFamilyName().getBytes();
-        this.cq = column.getName().getBytes();
+        // for backward compatibility since older tables won't have columnQualifierBytes in their metadata
+        this.cq = column.getColumnQualifierBytes() != null ? column.getColumnQualifierBytes() : column.getName().getBytes();
         this.displayName = displayName;
     }
 
+    public KeyValueColumnExpression(PDatum column, byte[] cf, byte[] cq) {
+        super(column);
+        this.cf = cf;
+        this.cq = cq;
+    }
+
     public byte[] getColumnFamily() {
         return cf;
     }
-
-    public byte[] getColumnName() {
+    
+    public byte[] getColumnQualifier() {
         return cq;
     }
 
@@ -120,7 +125,8 @@ public class KeyValueColumnExpression extends ColumnExpression {
     }
 
     @Override
-    public final <T> T accept(ExpressionVisitor<T> visitor) {
+    public <T> T accept(ExpressionVisitor<T> visitor) {
         return visitor.visit(this);
     }
+
 }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
new file mode 100644
index 0000000..8c1e0b6
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellColumnExpression.java
@@ -0,0 +1,182 @@
+/*
+ * 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 static org.apache.phoenix.query.QueryConstants.SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.compile.CreateTableCompiler.ViewWhereExpressionVisitor;
+import org.apache.phoenix.expression.visitor.ExpressionVisitor;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnValueDecoder;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PDatum;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.SchemaUtil;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * 
+ * Class to access a column that is stored in a Cell that contains all
+ * columns for a given column family (stored in a serialized array).
+ *
+ */
+public class SingleCellColumnExpression extends KeyValueColumnExpression {
+    
+    private int decodedColumnQualifier;
+    private String arrayColDisplayName;
+    private KeyValueColumnExpression keyValueColumnExpression;
+    private QualifierEncodingScheme encodingScheme;
+    
+    public SingleCellColumnExpression() {
+    }
+    
+    public SingleCellColumnExpression(PDatum column, byte[] cf, byte[] cq, QualifierEncodingScheme encodingScheme) {
+        super(column, cf, SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES);
+        Preconditions.checkNotNull(encodingScheme);
+        Preconditions.checkArgument(encodingScheme != NON_ENCODED_QUALIFIERS);
+        this.decodedColumnQualifier = encodingScheme.decode(cq);
+        this.encodingScheme = encodingScheme;
+        setKeyValueExpression();
+    }
+    
+    public SingleCellColumnExpression(PColumn column, String displayName, QualifierEncodingScheme encodingScheme) {
+        super(column, column.getFamilyName().getBytes(), SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES);
+        Preconditions.checkNotNull(encodingScheme);
+        Preconditions.checkArgument(encodingScheme != NON_ENCODED_QUALIFIERS);
+        this.arrayColDisplayName = displayName;
+        this.decodedColumnQualifier = encodingScheme.decode(column.getColumnQualifierBytes());
+        this.encodingScheme = encodingScheme;
+        setKeyValueExpression();
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+    	if (!super.evaluate(tuple, ptr)) {
+            return false;
+        } else if (ptr.getLength() == 0) { 
+        	return true; 
+        }
+    	// the first position is reserved and we offset maxEncodedColumnQualifier by ENCODED_CQ_COUNTER_INITIAL_VALUE (which is the minimum encoded column qualifier)
+    	int index = decodedColumnQualifier-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
+    	byte serializedImmutableStorageScheme = ptr.get()[ptr.getOffset() + ptr.getLength() - Bytes.SIZEOF_BYTE];
+    	ImmutableStorageScheme immutableStorageScheme = ImmutableStorageScheme.fromSerializedValue(serializedImmutableStorageScheme);
+        // Given a ptr to the entire array, set ptr to point to a particular element within that array
+    	ColumnValueDecoder encoderDecoder = immutableStorageScheme.getDecoder();
+    	return encoderDecoder.decode(ptr, index);
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        this.decodedColumnQualifier = WritableUtils.readVInt(input);
+        this.encodingScheme = QualifierEncodingScheme.values()[WritableUtils.readVInt(input)];
+        setKeyValueExpression();
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeVInt(output, decodedColumnQualifier);
+        WritableUtils.writeVInt(output, encodingScheme.ordinal());
+    }
+    
+    public KeyValueColumnExpression getKeyValueExpression() {
+        return keyValueColumnExpression;
+    }
+    
+    private void setKeyValueExpression() {
+        final boolean isNullable = isNullable();
+        final SortOrder sortOrder = getSortOrder();
+        final Integer scale = getScale();
+        final Integer maxLength = getMaxLength();
+        final PDataType datatype = getDataType();
+    	this.keyValueColumnExpression = new KeyValueColumnExpression(new PDatum() {
+			@Override
+			public boolean isNullable() {
+				return isNullable;
+			}
+			
+			@Override
+			public SortOrder getSortOrder() {
+				return sortOrder;
+			}
+			
+			@Override
+			public Integer getScale() {
+				return scale;
+			}
+			
+			@Override
+			public Integer getMaxLength() {
+				return maxLength;
+			}
+			
+			@Override
+			public PDataType getDataType() {
+				return datatype;
+			}
+		}, getColumnFamily(), getPositionInArray());
+    }
+    
+    @Override
+    public String toString() {
+        if (arrayColDisplayName == null) {
+            arrayColDisplayName = SchemaUtil.getColumnDisplayName(getColumnFamily(), getColumnQualifier());
+        }
+        return arrayColDisplayName;
+    }
+    
+    public byte[] getPositionInArray() {
+        return encodingScheme.encode(decodedColumnQualifier);
+    }
+    
+    @Override
+    public <T> T accept(ExpressionVisitor<T> visitor) {
+        //FIXME: this is ugly but can't think of a good solution.
+        if (visitor instanceof ViewWhereExpressionVisitor) {
+            return visitor.visit(this);
+        } else {
+            return super.accept(visitor);
+        }
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj.getClass() != SingleCellColumnExpression.class) return false;
+        return keyValueColumnExpression.equals(((SingleCellColumnExpression)obj).getKeyValueExpression());
+    }
+
+    @Override
+    public int hashCode() {
+        return keyValueColumnExpression.hashCode();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellConstructorExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellConstructorExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellConstructorExpression.java
new file mode 100644
index 0000000..48485be
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/SingleCellConstructorExpression.java
@@ -0,0 +1,102 @@
+/*
+ * 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 java.util.List;
+
+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.ColumnValueEncoder;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
+
+/**
+ * Expression used to create a single cell containing all the column values for a column family
+ */
+public class SingleCellConstructorExpression extends BaseCompoundExpression {
+    
+    private ImmutableStorageScheme immutableStorageScheme;
+    
+    public SingleCellConstructorExpression(ImmutableStorageScheme immutableStorageScheme, List<Expression> children) {
+        super(children);
+        this.immutableStorageScheme = immutableStorageScheme;
+    }
+
+    @Override
+    public PDataType getDataType() {
+        return PVarbinary.INSTANCE;
+    }
+
+    @Override
+    public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+        ColumnValueEncoder encoderDecoder = immutableStorageScheme.getEncoder(children.size());
+        for (int i=0; i < children.size(); i++) {
+            Expression child = children.get(i);
+            if (!child.evaluate(tuple, ptr)) {
+                encoderDecoder.appendAbsentValue();
+            } else {
+                encoderDecoder.appendValue(ptr.get(), ptr.getOffset(), ptr.getLength());
+            }
+        }
+        byte[] bytes = encoderDecoder.encode();
+        ptr.set(bytes, 0, bytes.length);
+        return true;
+    }
+
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        super.readFields(input);
+        this.immutableStorageScheme = WritableUtils.readEnum(input, ImmutableStorageScheme.class);
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        super.write(output);
+        WritableUtils.writeEnum(output, immutableStorageScheme);
+    }
+    
+    @Override
+    public boolean requiresFinalEvaluation() {
+        return true;
+    }
+    
+    @Override
+    public String toString() {
+        StringBuilder buf = new StringBuilder("[");
+        if (children.size()==0)
+            return buf.append("]").toString();
+        for (int i = 0; i < children.size() - 1; i++) {
+            buf.append(children.get(i) + ",");
+        }
+        buf.append(children.get(children.size()-1) + "]");
+        return buf.toString();
+    }
+
+    @Override
+    public final <T> T accept(ExpressionVisitor<T> visitor) {
+        List<T> l = acceptChildren(visitor, visitor.visitEnter(this));
+        T t = visitor.visitLeave(this, l);
+        if (t == null) {
+            t = visitor.defaultReturn(this, l);
+        }
+        return t;
+    }
+
+    public SingleCellConstructorExpression clone(List<Expression> children) {
+        return new SingleCellConstructorExpression(immutableStorageScheme, children);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayElemRefExpression.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayElemRefExpression.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayElemRefExpression.java
index 6631e70..06bbced 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayElemRefExpression.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayElemRefExpression.java
@@ -27,7 +27,7 @@ import org.apache.phoenix.expression.BaseCompoundExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.schema.tuple.Tuple;
-import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
 import org.apache.phoenix.schema.types.PDataType;
 
 public class ArrayElemRefExpression extends BaseCompoundExpression {
@@ -48,7 +48,7 @@ public class ArrayElemRefExpression extends BaseCompoundExpression {
     @Override
     public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
         Expression arrayExpr = children.get(0);
-        return PArrayDataType.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(), getMaxLength());
+        return PArrayDataTypeDecoder.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(), getMaxLength());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java
index 7a23ef5..0f3c40c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/function/ArrayIndexFunction.java
@@ -24,9 +24,9 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.parse.FunctionParseNode.Argument;
 import org.apache.phoenix.parse.FunctionParseNode.BuiltInFunction;
 import org.apache.phoenix.parse.ParseException;
+import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
 import org.apache.phoenix.schema.types.PBinaryArray;
 import org.apache.phoenix.schema.types.PInteger;
-import org.apache.phoenix.schema.types.PArrayDataType;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarbinaryArray;
 import org.apache.phoenix.schema.SortOrder;
@@ -61,7 +61,7 @@ public class ArrayIndexFunction extends ScalarFunction {
 			throw new ParseException("Index cannot be negative :" + index);
 		}
 		Expression arrayExpr = children.get(0);
-		return PArrayDataType.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(),
+		return PArrayDataTypeDecoder.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(),
         getMaxLength());
 	}
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
index af5bc2b..f2ed97b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/util/regex/JONIPattern.java
@@ -22,7 +22,7 @@ import java.util.List;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.schema.types.PArrayDataType.PArrayDataTypeBytesArrayBuilder;
+import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
@@ -159,8 +159,8 @@ public class JONIPattern extends AbstractBasePattern implements AbstractBaseSpli
     private boolean
             split(byte[] srcBytes, int srcOffset, int srcLen, ImmutableBytesWritable outPtr) {
         SortOrder sortOrder = SortOrder.ASC;
-        PArrayDataTypeBytesArrayBuilder builder =
-                new PArrayDataTypeBytesArrayBuilder(PVarchar.INSTANCE, sortOrder);
+        PArrayDataTypeEncoder builder =
+                new PArrayDataTypeEncoder(PVarchar.INSTANCE, sortOrder);
         int srcRange = srcOffset + srcLen;
         Matcher matcher = pattern.matcher(srcBytes, 0, srcRange);
         int cur = srcOffset;
@@ -168,31 +168,29 @@ public class JONIPattern extends AbstractBasePattern implements AbstractBaseSpli
         while (true) {
             int nextCur = matcher.search(cur, srcRange, Option.DEFAULT);
             if (nextCur < 0) {
-                append = builder.appendElem(srcBytes, cur, srcRange - cur);
-                if (!append) return false;
+                builder.appendValue(srcBytes, cur, srcRange - cur);
                 break;
             }
 
             // To handle the following case, which adds null at first.
             // REGEXP_SPLIT("12ONE34TWO56THREE78","[0-9]+")={null, "ONE", "TWO", "THREE", null}
             if (cur == matcher.getBegin()) {
-                builder.appendElem(srcBytes, cur, 0);
+                builder.appendValue(srcBytes, cur, 0);
             }
 
             if (cur < matcher.getBegin()) {
-                append = builder.appendElem(srcBytes, cur, matcher.getBegin() - cur);
-                if (!append) return false;
+                builder.appendValue(srcBytes, cur, matcher.getBegin() - cur);
             }
             cur = matcher.getEnd();
 
             // To handle the following case, which adds null at last.
             // REGEXP_SPLIT("12ONE34TWO56THREE78","[0-9]+")={null, "ONE", "TWO", "THREE", null}
             if (cur == srcRange) {
-                builder.appendElem(srcBytes, cur, 0);
+                builder.appendValue(srcBytes, cur, 0);
                 break;
             }
         }
-        byte[] bytes = builder.getBytesAndClose(SortOrder.ASC);
+        byte[] bytes = builder.encode();
         if (bytes == null) return false;
         outPtr.set(bytes);
         return true;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/BaseExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/BaseExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/BaseExpressionVisitor.java
index 8e8b32d..d79b546 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/BaseExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/BaseExpressionVisitor.java
@@ -36,6 +36,7 @@ import org.apache.phoenix.expression.MultiplyExpression;
 import org.apache.phoenix.expression.NotExpression;
 import org.apache.phoenix.expression.OrExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.expression.StringConcatExpression;
 import org.apache.phoenix.expression.SubtractExpression;
 import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
@@ -142,6 +143,11 @@ public abstract class BaseExpressionVisitor<E> implements ExpressionVisitor<E> {
     }
     
     @Override
+    public Iterator<Expression> visitEnter(SingleCellConstructorExpression node) {
+        return null;
+    }
+    
+    @Override
     public Iterator<Expression> visitEnter(ModulusExpression modulusExpression) {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..e47fb64 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
@@ -41,6 +41,8 @@ import org.apache.phoenix.expression.OrExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.expression.StringConcatExpression;
 import org.apache.phoenix.expression.SubtractExpression;
 import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
@@ -80,6 +82,11 @@ public abstract class CloneExpressionVisitor extends TraverseAllExpressionVisito
     public Expression visit(KeyValueColumnExpression node) {
         return node;
     }
+    
+    @Override
+    public Expression visit(SingleCellColumnExpression node) {
+        return node;
+    }
 
     @Override
     public Expression visit(ProjectedColumnExpression node) {
@@ -182,6 +189,11 @@ public abstract class CloneExpressionVisitor extends TraverseAllExpressionVisito
     public Expression visitLeave(ArrayConstructorExpression node, List<Expression> l) {
         return isCloneNode(node, l) ? node.clone(l) : node;
     }
+    
+    @Override
+    public Expression visitLeave(SingleCellConstructorExpression node, List<Expression> l) {
+        return isCloneNode(node, l) ? node.clone(l) : node;
+    }
 
     @Override
     public Expression visitLeave(StringConcatExpression node, List<Expression> l) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java
index 1aeb9a9..9a56e36 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/expression/visitor/CloneNonDeterministicExpressionVisitor.java
@@ -24,6 +24,7 @@ import org.apache.phoenix.expression.Expression;
 
 public class CloneNonDeterministicExpressionVisitor extends CloneExpressionVisitor {
 
+    @Override
     public boolean isCloneNode(Expression node, List<Expression> children) {
         return Determinism.PER_INVOCATION.compareTo(node.getDeterminism()) <= 0;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..5936dc7 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
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.phoenix.compile.SequenceValueExpression;
 import org.apache.phoenix.expression.AddExpression;
 import org.apache.phoenix.expression.AndExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
 import org.apache.phoenix.expression.ArrayConstructorExpression;
 import org.apache.phoenix.expression.CaseExpression;
 import org.apache.phoenix.expression.CoerceExpression;
@@ -42,6 +43,7 @@ import org.apache.phoenix.expression.OrExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.expression.StringConcatExpression;
 import org.apache.phoenix.expression.SubtractExpression;
 import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
@@ -109,10 +111,14 @@ public interface ExpressionVisitor<E> {
     public Iterator<Expression> visitEnter(ArrayConstructorExpression node);
     public E visitLeave(ArrayConstructorExpression node, List<E> l);
     
+    public Iterator<Expression> visitEnter(SingleCellConstructorExpression node);
+    public E visitLeave(SingleCellConstructorExpression node, List<E> l);
+    
     public E visit(CorrelateVariableFieldAccessExpression node);
     public E visit(LiteralExpression node);
     public E visit(RowKeyColumnExpression node);
     public E visit(KeyValueColumnExpression node);
+    public E visit(SingleCellColumnExpression node);
     public E visit(ProjectedColumnExpression node);
     public E visit(SequenceValueExpression node);
     

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..f5615be 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
@@ -28,7 +28,6 @@ import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
 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;
@@ -41,6 +40,8 @@ import org.apache.phoenix.expression.OrExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.expression.StringConcatExpression;
 import org.apache.phoenix.expression.SubtractExpression;
 import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
@@ -121,6 +122,11 @@ public class StatelessTraverseAllExpressionVisitor<E> extends TraverseAllExpress
     }
     
     @Override
+    public E visit(SingleCellColumnExpression node) {
+        return null;
+    }
+    
+    @Override
     public E visit(ProjectedColumnExpression node) {
         return null;
     }
@@ -164,6 +170,11 @@ public class StatelessTraverseAllExpressionVisitor<E> extends TraverseAllExpress
     public E visitLeave(ArrayConstructorExpression node, List<E> l) {
         return null;
     }
+    
+    @Override
+    public E visitLeave(SingleCellConstructorExpression node, List<E> l) {
+        return null;
+    }
 
     @Override
     public E visitLeave(ModulusExpression node, List<E> l) {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..7f447b3 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
@@ -28,7 +28,6 @@ import org.apache.phoenix.expression.CoerceExpression;
 import org.apache.phoenix.expression.ComparisonExpression;
 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;
@@ -41,6 +40,8 @@ import org.apache.phoenix.expression.OrExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
 import org.apache.phoenix.expression.RowValueConstructorExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.expression.StringConcatExpression;
 import org.apache.phoenix.expression.SubtractExpression;
 import org.apache.phoenix.expression.function.ArrayAnyComparisonExpression;
@@ -114,6 +115,11 @@ public class StatelessTraverseNoExpressionVisitor<E> extends TraverseNoExpressio
     public E visit(RowKeyColumnExpression node) {
         return null;
     }
+    
+    @Override
+    public E visit(SingleCellColumnExpression node) {
+        return null;
+    }
 
     @Override
     public E visit(KeyValueColumnExpression node) {
@@ -164,6 +170,11 @@ public class StatelessTraverseNoExpressionVisitor<E> extends TraverseNoExpressio
     public E visitLeave(ArrayConstructorExpression node, List<E> l) {
         return null;
     }
+    
+    @Override
+    public E visitLeave(SingleCellConstructorExpression node, List<E> l) {
+        return null;
+    }
 
     @Override
     public E visitLeave(ModulusExpression node, List<E> l) {

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/filter/EncodedQualifiersColumnProjectionFilter.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/filter/EncodedQualifiersColumnProjectionFilter.java b/phoenix-core/src/main/java/org/apache/phoenix/filter/EncodedQualifiersColumnProjectionFilter.java
new file mode 100644
index 0000000..cfacb4f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/filter/EncodedQualifiersColumnProjectionFilter.java
@@ -0,0 +1,151 @@
+/*
+ * 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.filter;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_EMPTY_COLUMN_BYTES;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.filter.FilterBase;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Writables;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+
+import com.google.common.base.Predicate;
+import com.google.common.collect.Iterables;
+
+public class EncodedQualifiersColumnProjectionFilter extends FilterBase implements Writable {
+
+    private byte[] emptyCFName;
+    private BitSet trackedColumns;
+    private QualifierEncodingScheme encodingScheme;
+    private Set<byte[]> conditionOnlyCfs;
+    
+    public EncodedQualifiersColumnProjectionFilter() {}
+
+    public EncodedQualifiersColumnProjectionFilter(byte[] emptyCFName, BitSet trackedColumns, Set<byte[]> conditionCfs, QualifierEncodingScheme encodingScheme) {
+        checkArgument(encodingScheme != NON_ENCODED_QUALIFIERS, "Filter can only be used for encoded qualifiers");
+        this.emptyCFName = emptyCFName;
+        this.trackedColumns = trackedColumns;
+        this.encodingScheme = encodingScheme;
+        this.conditionOnlyCfs = conditionCfs;
+    }
+
+    @Override
+    public void readFields(DataInput input) throws IOException {
+        this.emptyCFName = WritableUtils.readCompressedByteArray(input);
+        int bitsetLongArraySize = WritableUtils.readVInt(input);
+        long[] bitsetLongArray = new long[bitsetLongArraySize];
+        for (int i = 0; i < bitsetLongArraySize; i++) {
+            bitsetLongArray[i] = WritableUtils.readVLong(input);
+        }
+        this.trackedColumns = BitSet.valueOf(bitsetLongArray);
+        this.encodingScheme = QualifierEncodingScheme.values()[WritableUtils.readVInt(input)];
+        int conditionOnlyCfsSize = WritableUtils.readVInt(input);
+        this.conditionOnlyCfs = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
+        while (conditionOnlyCfsSize > 0) {
+            this.conditionOnlyCfs.add(WritableUtils.readCompressedByteArray(input));
+            conditionOnlyCfsSize--;
+        }
+    }
+
+    @Override
+    public void write(DataOutput output) throws IOException {
+        WritableUtils.writeCompressedByteArray(output, this.emptyCFName);
+        long[] longArrayOfBitSet = trackedColumns.toLongArray();
+        WritableUtils.writeVInt(output, longArrayOfBitSet.length);
+        for (Long l : longArrayOfBitSet) {
+            WritableUtils.writeVLong(output, l);
+        }
+        WritableUtils.writeVInt(output, encodingScheme.ordinal());
+        WritableUtils.writeVInt(output, this.conditionOnlyCfs.size());
+        for (byte[] f : this.conditionOnlyCfs) {
+            WritableUtils.writeCompressedByteArray(output, f);
+        }
+    }
+
+    @Override
+    public byte[] toByteArray() throws IOException {
+        return Writables.getBytes(this);
+    }
+    
+    public static EncodedQualifiersColumnProjectionFilter parseFrom(final byte [] pbBytes) throws DeserializationException {
+        try {
+            return (EncodedQualifiersColumnProjectionFilter)Writables.getWritable(pbBytes, new EncodedQualifiersColumnProjectionFilter());
+        } catch (IOException e) {
+            throw new DeserializationException(e);
+        }
+    }
+    
+    @Override
+    public void filterRowCells(List<Cell> kvs) throws IOException {
+        if (kvs.isEmpty()) return;
+        Cell firstKV = kvs.get(0);
+        Iterables.removeIf(kvs, new Predicate<Cell>() {
+            @Override
+            public boolean apply(Cell kv) {
+                int qualifier = encodingScheme.decode(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength());
+                return !trackedColumns.get(qualifier);
+            }
+        });
+        if (kvs.isEmpty()) {
+            kvs.add(new KeyValue(firstKV.getRowArray(), firstKV.getRowOffset(), firstKV.getRowLength(),
+                    this.emptyCFName, 0, this.emptyCFName.length, ENCODED_EMPTY_COLUMN_BYTES, 0,
+                    ENCODED_EMPTY_COLUMN_BYTES.length, HConstants.LATEST_TIMESTAMP, Type.Maximum, null, 0, 0));
+        }
+    }
+
+    @Override
+    public boolean hasFilterRow() {
+        return true;
+    }
+
+    @Override
+    public boolean isFamilyEssential(byte[] name) {
+        return conditionOnlyCfs.isEmpty() || this.conditionOnlyCfs.contains(name);
+    }
+
+    @Override
+    public String toString() {
+        return "";
+    }
+    
+    @Override
+    public ReturnCode filterKeyValue(Cell ignored) throws IOException {
+      return ReturnCode.INCLUDE_AND_NEXT_COL;
+    }
+    
+    interface ColumnTracker {
+        
+    }
+}


[04/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTable.java
index 01e8afe..91a41a3 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,13 +17,32 @@
  */
 package org.apache.phoenix.schema;
 
+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.util.EncodedColumnsUtil.isReservedColumnQualifier;
+
+import java.io.DataOutputStream;
+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;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;
+import org.apache.phoenix.schema.types.PArrayDataTypeEncoder;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+import com.google.common.annotations.VisibleForTesting;
 
 
 /**
@@ -129,7 +148,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 +172,318 @@ public interface PTable extends PMetaDataEntity {
             return LinkType.values()[serializedValue-1];
         }
     }
+    
+    public enum ImmutableStorageScheme implements ColumnValueEncoderDecoderSupplier {
+        ONE_CELL_PER_COLUMN((byte)1) {
+            @Override
+            public ColumnValueEncoder getEncoder(int numElements) {
+                throw new UnsupportedOperationException();
+            }
+            
+            @Override
+            public ColumnValueDecoder getDecoder() {
+                throw new UnsupportedOperationException();
+            }
+        },
+        // stores a single cell per column family that contains all serialized column values
+        SINGLE_CELL_ARRAY_WITH_OFFSETS((byte)2) {
+            @Override
+            public ColumnValueEncoder getEncoder(int numElements) {
+                PDataType type = PVarbinary.INSTANCE;
+                int estimatedSize = PArrayDataType.estimateSize(numElements, type);
+                TrustedByteArrayOutputStream byteStream = new TrustedByteArrayOutputStream(estimatedSize);
+                DataOutputStream oStream = new DataOutputStream(byteStream);
+                return new PArrayDataTypeEncoder(byteStream, oStream, numElements, type, SortOrder.ASC, false, PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION);
+            }
+            
+            @Override
+            public ColumnValueDecoder getDecoder() {
+                return new PArrayDataTypeDecoder();
+            }
+        };
+
+        private final byte serializedValue;
+        
+        private ImmutableStorageScheme(byte serializedValue) {
+            this.serializedValue = serializedValue;
+        }
+
+        public byte getSerializedMetadataValue() {
+            return this.serializedValue;
+        }
+
+        public static ImmutableStorageScheme fromSerializedValue(byte serializedValue) {
+            if (serializedValue < 1 || serializedValue > ImmutableStorageScheme.values().length) {
+                return null;
+            }
+            return ImmutableStorageScheme.values()[serializedValue-1];
+        }
+
+    }
+    
+    interface ColumnValueEncoderDecoderSupplier {
+        ColumnValueEncoder getEncoder(int numElements);
+        ColumnValueDecoder getDecoder();
+    }
+    
+    public enum QualifierEncodingScheme implements QualifierEncoderDecoder {
+        NON_ENCODED_QUALIFIERS((byte)0, null) {
+            @Override
+            public byte[] encode(int value) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public int decode(byte[] bytes) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public int decode(byte[] bytes, int offset, int length) {
+                throw new UnsupportedOperationException();
+            }
+            
+            @Override
+            public String toString() {
+                return name();
+            }
+        },
+        ONE_BYTE_QUALIFIERS((byte)1, 255) {
+            private final int c = Math.abs(Byte.MIN_VALUE);
+            
+            @Override
+            public byte[] encode(int value) {
+                if (isReservedColumnQualifier(value)) {
+                    return FOUR_BYTE_QUALIFIERS.encode(value);
+                }
+                if (value < 0 || value > maxQualifier) {
+                    throw new QualifierOutOfRangeException(0, maxQualifier);
+                }
+                return new byte[]{(byte)(value - c)};
+            }
+
+            @Override
+            public int decode(byte[] bytes) {
+                if (bytes.length == 4) {
+                    return getReservedQualifier(bytes);
+                }
+                if (bytes.length != 1) {
+                    throw new InvalidQualifierBytesException(1, bytes.length);
+                }
+                return bytes[0] + c;
+            }
+
+            @Override
+            public int decode(byte[] bytes, int offset, int length) {
+                if (length == 4) {
+                    return getReservedQualifier(bytes, offset, length);
+                }
+                if (length != 1) {
+                    throw new InvalidQualifierBytesException(1, length);
+                }
+                return bytes[offset] + c;
+            }
+            
+            @Override
+            public String toString() {
+                return name();
+            }
+        },
+        TWO_BYTE_QUALIFIERS((byte)2, 65535) {
+            private final int c = Math.abs(Short.MIN_VALUE);
+            
+            @Override
+            public byte[] encode(int value) {
+                if (isReservedColumnQualifier(value)) {
+                    return FOUR_BYTE_QUALIFIERS.encode(value);
+                }
+                if (value < 0 || value > maxQualifier) {
+                    throw new QualifierOutOfRangeException(0, maxQualifier);
+                }
+                return Bytes.toBytes((short)(value - c));
+            }
+
+            @Override
+            public int decode(byte[] bytes) {
+                if (bytes.length == 4) {
+                    return getReservedQualifier(bytes);
+                }
+                if (bytes.length != 2) {
+                    throw new InvalidQualifierBytesException(2, bytes.length);
+                }
+                return Bytes.toShort(bytes) + c;
+            }
+
+            @Override
+            public int decode(byte[] bytes, int offset, int length) {
+                if (length == 4) {
+                    return getReservedQualifier(bytes, offset, length);
+                }
+                if (length != 2) {
+                    throw new InvalidQualifierBytesException(2, length);
+                }
+                return Bytes.toShort(bytes, offset, length) + c;
+            }
+            
+            @Override
+            public String toString() {
+                return name();
+            }
+        },
+        THREE_BYTE_QUALIFIERS((byte)3, 16777215) {
+            @Override
+            public byte[] encode(int value) {
+                if (isReservedColumnQualifier(value)) {
+                    return FOUR_BYTE_QUALIFIERS.encode(value);
+                }
+                if (value < 0 || value > maxQualifier) {
+                    throw new QualifierOutOfRangeException(0, maxQualifier);
+                }
+                byte[] arr = Bytes.toBytes(value);
+                return new byte[]{arr[1], arr[2], arr[3]};
+            }
+
+            @Override
+            public int decode(byte[] bytes) {
+                if (bytes.length == 4) {
+                    return getReservedQualifier(bytes);
+                }
+                if (bytes.length != 3) {
+                    throw new InvalidQualifierBytesException(2, bytes.length);
+                }
+                byte[] toReturn = new byte[4];
+                toReturn[1] = bytes[0];
+                toReturn[2] = bytes[1];
+                toReturn[3] = bytes[2];
+                return Bytes.toInt(toReturn);
+            }
+
+            @Override
+            public int decode(byte[] bytes, int offset, int length) {
+                if (length == 4) {
+                    return getReservedQualifier(bytes, offset, length);
+                }
+                if (length != 3) {
+                    throw new InvalidQualifierBytesException(3, length);
+                }
+                byte[] toReturn = new byte[4];
+                toReturn[1] = bytes[offset];
+                toReturn[2] = bytes[offset + 1];
+                toReturn[3] = bytes[offset + 2];
+                return Bytes.toInt(toReturn);
+            }
+            
+            @Override
+            public String toString() {
+                return name();
+            }
+        },
+        FOUR_BYTE_QUALIFIERS((byte)4, Integer.MAX_VALUE) {
+            @Override
+            public byte[] encode(int value) {
+                if (value < 0) {
+                    throw new QualifierOutOfRangeException(0, maxQualifier);
+                }
+                return Bytes.toBytes(value);
+            }
+
+            @Override
+            public int decode(byte[] bytes) {
+                if (bytes.length != 4) {
+                    throw new InvalidQualifierBytesException(4, bytes.length);
+                }
+                return Bytes.toInt(bytes);
+            }
+
+            @Override
+            public int decode(byte[] bytes, int offset, int length) {
+                if (length != 4) {
+                    throw new InvalidQualifierBytesException(4, length);
+                }
+                return Bytes.toInt(bytes, offset, length);
+            }
+            
+            @Override
+            public String toString() {
+                return name();
+            }
+        };
+        
+        final byte metadataValue;
+        final Integer maxQualifier;
+        
+        public byte getSerializedMetadataValue() {
+            return this.metadataValue;
+        }
+
+        public static QualifierEncodingScheme fromSerializedValue(byte serializedValue) {
+            if (serializedValue < 0 || serializedValue >= QualifierEncodingScheme.values().length) {
+                return null;
+            }
+            return QualifierEncodingScheme.values()[serializedValue];
+        }
+        
+        @Override
+        public Integer getMaxQualifier() {
+            return maxQualifier;
+        }
+
+        private QualifierEncodingScheme(byte serializedMetadataValue, Integer maxQualifier) {
+            this.metadataValue = serializedMetadataValue;
+            this.maxQualifier = maxQualifier;
+        }
+        
+        @VisibleForTesting
+        public static class QualifierOutOfRangeException extends RuntimeException {
+            public QualifierOutOfRangeException(int minQualifier, int maxQualifier) {
+                super("Qualifier out of range (" + minQualifier + ", " + maxQualifier + ")"); 
+            }
+        }
+        
+        @VisibleForTesting
+        public static class InvalidQualifierBytesException extends RuntimeException {
+            public InvalidQualifierBytesException(int expectedLength, int actualLength) {
+                super("Invalid number of qualifier bytes. Expected length: " + expectedLength + ". Actual: " + actualLength);
+            }
+        }
+
+        /**
+         * We generate our column qualifiers in the reserved range 0-10 using the FOUR_BYTE_QUALIFIERS
+         * encoding. When adding Cells corresponding to the reserved qualifiers to the
+         * EncodedColumnQualifierCells list, we need to make sure that we use the FOUR_BYTE_QUALIFIERS
+         * scheme to decode the correct int value.
+         */
+        private static int getReservedQualifier(byte[] bytes) {
+            checkArgument(bytes.length == 4);
+            int number = FOUR_BYTE_QUALIFIERS.decode(bytes);
+            if (!isReservedColumnQualifier(number)) {
+                throw new InvalidQualifierBytesException(4, bytes.length);
+            }
+            return number;
+        }
+
+        /**
+         * We generate our column qualifiers in the reserved range 0-10 using the FOUR_BYTE_QUALIFIERS
+         * encoding. When adding Cells corresponding to the reserved qualifiers to the
+         * EncodedColumnQualifierCells list, we need to make sure that we use the FOUR_BYTE_QUALIFIERS
+         * scheme to decode the correct int value.
+         */
+        private static int getReservedQualifier(byte[] bytes, int offset, int length) {
+            checkArgument(length == 4);
+            int number = FOUR_BYTE_QUALIFIERS.decode(bytes, offset, length);
+            if (!isReservedColumnQualifier(number)) {
+                throw new InvalidQualifierBytesException(4, length);
+            }
+            return number;
+        }
+    }
+    
+    interface QualifierEncoderDecoder {
+        byte[] encode(int value);
+        int decode(byte[] bytes);
+        int decode(byte[] bytes, int offset, int length);
+        Integer getMaxQualifier();
+    }
 
     long getTimeStamp();
     long getSequenceNumber();
@@ -208,7 +539,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 getColumnForColumnName(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 getColumnForColumnQualifier(byte[] cf, byte[] cq) throws ColumnNotFoundException, AmbiguousColumnException; 
     
     /**
      * Get the PK column with the given name.
@@ -345,7 +685,6 @@ public interface PTable extends PMetaDataEntity {
      */
     int getRowTimestampColPos();
     long getUpdateCacheFrequency();
-
     boolean isNamespaceMapped();
     
     /**
@@ -359,4 +698,94 @@ public interface PTable extends PMetaDataEntity {
      * you are also not allowed to delete the table  
      */
     boolean isAppendOnlySchema();
+    ImmutableStorageScheme getImmutableStorageScheme();
+    QualifierEncodingScheme getEncodingScheme();
+    EncodedCQCounter getEncodedCQCounter();
+    
+    /**
+     * Class to help track encoded column qualifier counters per column family.
+     */
+    public class EncodedCQCounter {
+        
+        private final Map<String, Integer> familyCounters = new HashMap<>();
+        
+        /**
+         * Copy constructor
+         * @param counterToCopy
+         * @return copy of the passed counter
+         */
+        public static EncodedCQCounter copy(EncodedCQCounter counterToCopy) {
+            EncodedCQCounter cqCounter = new EncodedCQCounter();
+            for (Entry<String, Integer> e : counterToCopy.values().entrySet()) {
+                cqCounter.setValue(e.getKey(), e.getValue());
+            }
+            return cqCounter;
+        }
+        
+        public static final EncodedCQCounter NULL_COUNTER = new EncodedCQCounter() {
+
+            @Override
+            public Integer getNextQualifier(String columnFamily) {
+                return null;
+            }
+
+            @Override
+            public void setValue(String columnFamily, Integer value) {
+            }
+
+            @Override
+            public boolean increment(String columnFamily) {
+                return false;
+            }
+
+            @Override
+            public Map<String, Integer> values() {
+                return Collections.emptyMap();
+            }
+
+        };
+        
+        /**
+         * Get the next qualifier to be used for the column family.
+         * This method also ends up initializing the counter if the
+         * column family already doesn't have one.
+         */
+        @Nullable
+        public Integer getNextQualifier(String columnFamily) {
+            Integer counter = familyCounters.get(columnFamily);
+            if (counter == null) {
+                counter = ENCODED_CQ_COUNTER_INITIAL_VALUE;
+                familyCounters.put(columnFamily, counter);
+            }
+            return counter;
+        }
+        
+        public void setValue(String columnFamily, Integer value) {
+            familyCounters.put(columnFamily, value);
+        }
+        
+        /**
+         * 
+         * @param columnFamily
+         * @return true if the counter was incremented, 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/d6a82e29/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 b4e0a06..d91ebcb 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
@@ -25,11 +25,15 @@ import java.io.IOException;
 import java.sql.DriverManager;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
+
+import javax.annotation.Nonnull;
 
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.HConstants;
@@ -41,11 +45,14 @@ import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.ExpressionCompiler;
 import org.apache.phoenix.compile.StatementContext;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.exception.DataExceedsCapacityException;
 import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.expression.SingleCellConstructorExpression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
 import org.apache.phoenix.index.IndexMaintainer;
@@ -63,6 +70,7 @@ import org.apache.phoenix.schema.types.PDouble;
 import org.apache.phoenix.schema.types.PFloat;
 import org.apache.phoenix.schema.types.PVarchar;
 import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
@@ -79,6 +87,7 @@ import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
+
 /**
  *
  * Base class for PTable implementors.  Provides abstraction for
@@ -106,7 +115,8 @@ public class PTableImpl implements PTable {
     private List<PColumnFamily> families;
     private Map<byte[], PColumnFamily> familyByBytes;
     private Map<String, PColumnFamily> familyByString;
-    private ListMultimap<String,PColumn> columnsByName;
+    private ListMultimap<String, PColumn> columnsByName;
+    private Map<KVColumnFamilyQualifier, PColumn> kvColumnsByQualifiers;
     private PName pkName;
     private Integer bucketNum;
     private RowKeySchema rowKeySchema;
@@ -138,14 +148,18 @@ public class PTableImpl implements PTable {
     private boolean isNamespaceMapped;
     private String autoPartitionSeqName;
     private boolean isAppendOnlySchema;
+    private ImmutableStorageScheme immutableStorageScheme;
+    private QualifierEncodingScheme qualifierEncodingScheme;
+    private EncodedCQCounter encodedCQCounter;
 
     public PTableImpl() {
         this.indexes = Collections.emptyList();
         this.physicalNames = Collections.emptyList();
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
     }
-
-    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped) { // For base table of mapped VIEW
+    
+    // Constructor used at table creation time
+    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped) {
         Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
         this.tenantId = tenantId;
         this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
@@ -169,8 +183,36 @@ public class PTableImpl implements PTable {
         this.isNamespaceMapped = isNamespaceMapped;
     }
     
+    public PTableImpl(PName tenantId, String schemaName, String tableName, long timestamp, List<PColumnFamily> families, boolean isNamespaceMapped, ImmutableStorageScheme storageScheme, QualifierEncodingScheme encodingScheme) { // For base table of mapped VIEW
+        Preconditions.checkArgument(tenantId==null || tenantId.getBytes().length > 0); // tenantId should be null or not empty
+        this.tenantId = tenantId;
+        this.name = PNameFactory.newName(SchemaUtil.getTableName(schemaName, tableName));
+        this.key = new PTableKey(tenantId, name.getString());
+        this.schemaName = PNameFactory.newName(schemaName);
+        this.tableName = PNameFactory.newName(tableName);
+        this.type = PTableType.VIEW;
+        this.viewType = ViewType.MAPPED;
+        this.timeStamp = timestamp;
+        this.pkColumns = this.allColumns = Collections.emptyList();
+        this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
+        this.indexes = Collections.emptyList();
+        this.familyByBytes = Maps.newHashMapWithExpectedSize(families.size());
+        this.familyByString = Maps.newHashMapWithExpectedSize(families.size());
+        for (PColumnFamily family : families) {
+            familyByBytes.put(family.getName().getBytes(), family);
+            familyByString.put(family.getName().getString(), family);
+        }
+        this.families = families;
+        this.physicalNames = Collections.emptyList();
+        this.isNamespaceMapped = isNamespaceMapped;
+        this.immutableStorageScheme = storageScheme;
+        this.qualifierEncodingScheme = encodingScheme;
+    }
+    
+    // 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, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
+            EncodedCQCounter encodedCQCounter) throws SQLException {
         this.pkColumns = this.allColumns = Collections.emptyList();
         this.rowKeySchema = RowKeySchema.EMPTY_SCHEMA;
         this.indexes = Collections.emptyList();
@@ -184,7 +226,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, qualifierEncodingScheme, encodedCQCounter);
     }
 
     public PTableImpl(long timeStamp) { // For delete marker
@@ -228,7 +270,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
         }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, List<PTable> indexes, PName parentSchemaName, String viewStatement) throws SQLException {
@@ -238,7 +280,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, Collection<PColumn> columns) throws SQLException {
@@ -248,7 +290,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
     
     public static PTableImpl makePTable(PTable table, Collection<PColumn> columns, PName defaultFamily) throws SQLException {
@@ -258,7 +300,7 @@ public class PTableImpl implements PTable {
                 table.getIndexes(), table.isImmutableRows(), table.getPhysicalNames(), defaultFamily, 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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns) throws SQLException {
@@ -268,7 +310,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows) throws SQLException {
@@ -278,7 +320,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
     
     public static PTableImpl makePTable(PTable table, long timeStamp, long sequenceNumber, Collection<PColumn> columns, boolean isImmutableRows, boolean isWalDisabled,
@@ -289,7 +331,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
     
     public static PTableImpl makePTable(PTable table, PIndexState state) throws SQLException {
@@ -300,7 +342,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table, boolean rowKeyOrderOptimizable) throws SQLException {
@@ -311,7 +353,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PTable table) throws SQLException {
@@ -322,7 +364,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.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -331,12 +373,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, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 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, qualifierEncodingScheme, encodedCQCounter);
     }
 
     public static PTableImpl makePTable(PName tenantId, PName schemaName, PName tableName, PTableType type,
@@ -346,13 +388,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, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 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, qualifierEncodingScheme, encodedCQCounter);
     }
 
     private PTableImpl(PName tenantId, PName schemaName, PName tableName, PTableType type, PIndexState state,
@@ -361,11 +403,13 @@ 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, ImmutableStorageScheme storageScheme, 
+            QualifierEncodingScheme qualifierEncodingScheme, 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, 
+                qualifierEncodingScheme, encodedCQCounter);
     }
     
     @Override
@@ -399,7 +443,8 @@ 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, ImmutableStorageScheme storageScheme, QualifierEncodingScheme qualifierEncodingScheme, 
+            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 +
@@ -435,10 +480,13 @@ public class PTableImpl implements PTable {
         this.isNamespaceMapped = isNamespaceMapped;
         this.autoPartitionSeqName = autoPartitionSeqName;
         this.isAppendOnlySchema = isAppendOnlySchema;
+        this.immutableStorageScheme = storageScheme;
+        this.qualifierEncodingScheme = qualifierEncodingScheme;
         List<PColumn> pkColumns;
         PColumn[] allColumns;
         
         this.columnsByName = ArrayListMultimap.create(columns.size(), 1);
+        this.kvColumnsByQualifiers = Maps.newHashMapWithExpectedSize(columns.size());
         int numPKColumns = 0;
         if (bucketNum != null) {
             // Add salt column to allColumns and pkColumns, but don't add to
@@ -464,11 +512,21 @@ 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);
                         }
                     }
                 }
             }
+            byte[] cq = column.getColumnQualifierBytes();
+            String cf = column.getFamilyName() != null ? column.getFamilyName().getString() : null;
+            if (cf != null && cq != null) {
+                KVColumnFamilyQualifier info = new KVColumnFamilyQualifier(cf, cq);
+                if (kvColumnsByQualifiers.get(info) != null) {
+                    throw new ColumnAlreadyExistsException(schemaName.getString(),
+                            name.getString(), columnName);
+                }
+                kvColumnsByQualifiers.put(info, column);
+            }
         }
         estimatedSize += SizedUtil.sizeOfMap(allColumns.length, SizedUtil.POINTER_SIZE, SizedUtil.sizeOfArrayList(1)); // for multi-map
 
@@ -528,7 +586,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());//, qualifierEncodingScheme);
             families[i] = family;
             familyByString.put(family.getName().getString(), family);
             familyByBytes.put(family.getName().getBytes(), family);
@@ -554,9 +612,9 @@ public class PTableImpl implements PTable {
         for (PName name : this.physicalNames) {
             estimatedSize += name.getEstimatedSize();
         }
-
         this.estimatedSize = estimatedSize;
         this.baseColumnCount = baseColumnCount;
+        this.encodedCQCounter = encodedCQCounter;
     }
 
     @Override
@@ -746,7 +804,7 @@ public class PTableImpl implements PTable {
     }
 
     @Override
-    public PColumn getColumn(String name) throws ColumnNotFoundException, AmbiguousColumnException {
+    public PColumn getColumnForColumnName(String name) throws ColumnNotFoundException, AmbiguousColumnException {
         List<PColumn> columns = columnsByName.get(name);
         int size = columns.size();
         if (size == 0) {
@@ -765,6 +823,22 @@ public class PTableImpl implements PTable {
         }
         return columns.get(0);
     }
+    
+    @Override
+    public PColumn getColumnForColumnQualifier(byte[] cf, byte[] cq) throws ColumnNotFoundException, AmbiguousColumnException {
+        Preconditions.checkNotNull(cq);
+        if (!EncodedColumnsUtil.usesEncodedColumnNames(this) || cf == null) {
+            String columnName = (String)PVarchar.INSTANCE.toObject(cq);
+            return getColumnForColumnName(columnName);
+        } else {
+            String family = (String)PVarchar.INSTANCE.toObject(cf);
+            PColumn col = kvColumnsByQualifiers.get(new KVColumnFamilyQualifier(family, cq));
+            if (col == null) {
+                throw new ColumnNotFoundException("No column found for column qualifier " + qualifierEncodingScheme.decode(cq));
+            }
+            return col;
+        }
+    }
 
     /**
      *
@@ -785,6 +859,8 @@ public class PTableImpl implements PTable {
         private Mutation deleteRow;
         private final long ts;
         private final boolean hasOnDupKey;
+        // map from column name to value 
+        private Map<PColumn, byte[]> columnToValueMap; 
 
         public PRowImpl(KeyValueBuilder kvBuilder, ImmutableBytesWritable key, long ts, Integer bucketNum, boolean hasOnDupKey) {
             this.kvBuilder = kvBuilder;
@@ -797,7 +873,7 @@ public class PTableImpl implements PTable {
                 this.keyPtr =  new ImmutableBytesPtr(key);
                 this.key = ByteUtil.copyKeyBytesIfNecessary(key);
             }
-
+            this.columnToValueMap = Maps.newHashMapWithExpectedSize(1);
             newMutations();
         }
 
@@ -819,13 +895,48 @@ 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 (immutableStorageScheme != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
+                    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) {
+                            int qualifier = qualifierEncodingScheme.decode(column.getColumnQualifierBytes());
+                            maxEncodedColumnQualifier = Math.max(maxEncodedColumnQualifier, qualifier);
+                        }
+                        Expression[] colValues = EncodedColumnsUtil.createColumnExpressionArray(maxEncodedColumnQualifier);
+                        for (PColumn column : columns) {
+                        	if (columnToValueMap.containsKey(column)) {
+                        	    int colIndex = qualifierEncodingScheme.decode(column.getColumnQualifierBytes())-QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE+1;
+                        	    colValues[colIndex] = new LiteralExpression(columnToValueMap.get(column));
+                        	}
+                        }
+                        
+                        List<Expression> children = Arrays.asList(colValues);
+                        // we use SingleCellConstructorExpression to serialize all the columns into a single byte[]
+                        SingleCellConstructorExpression singleCellConstructorExpression = new SingleCellConstructorExpression(immutableStorageScheme, children);
+                        ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+                        singleCellConstructorExpression.evaluate(null, ptr);
+                        ImmutableBytesPtr colFamilyPtr = new ImmutableBytesPtr(columnFamily);
+                        addQuietly(put, kvBuilder, kvBuilder.buildPut(keyPtr,
+                            colFamilyPtr, QueryConstants.SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES_PTR, 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);
@@ -854,7 +965,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 = column.getColumnQualifierBytes();
+            ImmutableBytesPtr qualifierPtr = new ImmutableBytesPtr(qualifier);
             PDataType<?> type = column.getDataType();
             // Check null, since some types have no byte representation for null
             if (byteValue == null) {
@@ -874,7 +986,7 @@ public class PTableImpl implements PTable {
                 // case of updates occurring due to the execution of the clause.
                 removeIfPresent(setValues, family, qualifier);
                 deleteQuietly(unsetValues, kvBuilder, kvBuilder.buildDeleteColumns(keyPtr, column
-                            .getFamilyName().getBytesPtr(), column.getName().getBytesPtr(), ts));
+                            .getFamilyName().getBytesPtr(), qualifierPtr, ts));
             } else {
                 ImmutableBytesWritable ptr = new ImmutableBytesWritable(byteValue);
                 Integer	maxLength = column.getMaxLength();
@@ -887,9 +999,17 @@ public class PTableImpl implements PTable {
                 ptr.set(byteValue);
                 type.pad(ptr, maxLength, sortOrder);
                 removeIfPresent(unsetValues, family, qualifier);
-                addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
-                        column.getFamilyName().getBytesPtr(), column.getName().getBytesPtr(),
+                // store all columns for a given column family in a single cell instead of one column per cell in order to improve write performance
+                // we don't need to do anything with unsetValues as it is only used when storeNulls is false, storeNulls is always true when storeColsInSingleCell is true
+                if (immutableStorageScheme == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS) {
+                    columnToValueMap.put(column, ptr.get());
+                }
+                else {
+                    removeIfPresent(unsetValues, family, qualifier);
+                    addQuietly(setValues, kvBuilder, kvBuilder.buildPut(keyPtr,
+                        column.getFamilyName().getBytesPtr(), qualifierPtr,
                         ts, ptr));
+                }
             }
         }
 
@@ -922,6 +1042,7 @@ public class PTableImpl implements PTable {
                 deleteRow.setDurability(Durability.SKIP_WAL);
             }
         }
+        
     }
 
     @Override
@@ -1082,116 +1203,136 @@ public class PTableImpl implements PTable {
     public IndexType getIndexType() {
         return indexType;
     }
-
+    
     /**
      * Construct a PTable instance from ProtoBuffered PTable instance
      * @param table
      */
     public static PTable createFromProto(PTableProtos.PTable table) {
-      PName tenantId = null;
-      if(table.hasTenantId()){
-        tenantId = PNameFactory.newName(table.getTenantId().toByteArray());
-      }
-      PName schemaName = PNameFactory.newName(table.getSchemaNameBytes().toByteArray());
-      PName tableName = PNameFactory.newName(table.getTableNameBytes().toByteArray());
-      PTableType tableType = PTableType.values()[table.getTableType().ordinal()];
-      PIndexState indexState = null;
-      if (table.hasIndexState()) {
-        indexState = PIndexState.fromSerializedValue(table.getIndexState());
-      }
-      Short viewIndexId = null;
-      if(table.hasViewIndexId()){
-    	  viewIndexId = (short)table.getViewIndexId();
-      }
-      IndexType indexType = IndexType.getDefault();
-      if(table.hasIndexType()){
-          indexType = IndexType.fromSerializedValue(table.getIndexType().toByteArray()[0]);
-      }
-      long sequenceNumber = table.getSequenceNumber();
-      long timeStamp = table.getTimeStamp();
-      long indexDisableTimestamp = table.getIndexDisableTimestamp();
-      PName pkName = null;
-      if (table.hasPkNameBytes()) {
-        pkName = PNameFactory.newName(table.getPkNameBytes().toByteArray());
-      }
-      int bucketNum = table.getBucketNum();
-      List<PColumn> columns = Lists.newArrayListWithExpectedSize(table.getColumnsCount());
-      for (PTableProtos.PColumn curPColumnProto : table.getColumnsList()) {
-        columns.add(PColumnImpl.createFromProto(curPColumnProto));
-      }
-      List<PTable> indexes = Lists.newArrayListWithExpectedSize(table.getIndexesCount());
-      for (PTableProtos.PTable curPTableProto : table.getIndexesList()) {
-        indexes.add(createFromProto(curPTableProto));
-      }
+        PName tenantId = null;
+        if(table.hasTenantId()){
+            tenantId = PNameFactory.newName(table.getTenantId().toByteArray());
+        }
+        PName schemaName = PNameFactory.newName(table.getSchemaNameBytes().toByteArray());
+        PName tableName = PNameFactory.newName(table.getTableNameBytes().toByteArray());
+        PTableType tableType = PTableType.values()[table.getTableType().ordinal()];
+        PIndexState indexState = null;
+        if (table.hasIndexState()) {
+            indexState = PIndexState.fromSerializedValue(table.getIndexState());
+        }
+        Short viewIndexId = null;
+        if(table.hasViewIndexId()){
+            viewIndexId = (short)table.getViewIndexId();
+        }
+        IndexType indexType = IndexType.getDefault();
+        if(table.hasIndexType()){
+            indexType = IndexType.fromSerializedValue(table.getIndexType().toByteArray()[0]);
+        }
+        long sequenceNumber = table.getSequenceNumber();
+        long timeStamp = table.getTimeStamp();
+        long indexDisableTimestamp = table.getIndexDisableTimestamp();
+        PName pkName = null;
+        if (table.hasPkNameBytes()) {
+            pkName = PNameFactory.newName(table.getPkNameBytes().toByteArray());
+        }
+        int bucketNum = table.getBucketNum();
+        List<PColumn> columns = Lists.newArrayListWithExpectedSize(table.getColumnsCount());
+        for (PTableProtos.PColumn curPColumnProto : table.getColumnsList()) {
+            columns.add(PColumnImpl.createFromProto(curPColumnProto));
+        }
+        List<PTable> indexes = Lists.newArrayListWithExpectedSize(table.getIndexesCount());
+        for (PTableProtos.PTable curPTableProto : table.getIndexesList()) {
+            indexes.add(createFromProto(curPTableProto));
+        }
 
-      boolean isImmutableRows = table.getIsImmutableRows();
-      PName parentSchemaName = null;
-      PName parentTableName = null;
-      if (table.hasParentNameBytes()) {
-        parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName((table.getParentNameBytes().toByteArray())));
-        parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(table.getParentNameBytes().toByteArray()));
-      }
-      PName defaultFamilyName = null;
-      if (table.hasDefaultFamilyName()) {
-        defaultFamilyName = PNameFactory.newName(table.getDefaultFamilyName().toByteArray());
-      }
-      boolean disableWAL = table.getDisableWAL();
-      boolean multiTenant = table.getMultiTenant();
-      boolean storeNulls = table.getStoreNulls();
-      boolean isTransactional = table.getTransactional();
-      ViewType viewType = null;
-      String viewStatement = null;
-      List<PName> physicalNames = Collections.emptyList();
-      if (tableType == PTableType.VIEW) {
-        viewType = ViewType.fromSerializedValue(table.getViewType().toByteArray()[0]);
-      }
-      if(table.hasViewStatement()){
-        viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
-      }
-      if (tableType == PTableType.VIEW || viewIndexId != null) {
-        physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
-        for(int i = 0; i < table.getPhysicalNamesCount(); i++){
-          physicalNames.add(PNameFactory.newName(table.getPhysicalNames(i).toByteArray()));
+        boolean isImmutableRows = table.getIsImmutableRows();
+        PName parentSchemaName = null;
+        PName parentTableName = null;
+        if (table.hasParentNameBytes()) {
+            parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName((table.getParentNameBytes().toByteArray())));
+            parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(table.getParentNameBytes().toByteArray()));
+        }
+        PName defaultFamilyName = null;
+        if (table.hasDefaultFamilyName()) {
+            defaultFamilyName = PNameFactory.newName(table.getDefaultFamilyName().toByteArray());
+        }
+        boolean disableWAL = table.getDisableWAL();
+        boolean multiTenant = table.getMultiTenant();
+        boolean storeNulls = table.getStoreNulls();
+        boolean isTransactional = table.getTransactional();
+        ViewType viewType = null;
+        String viewStatement = null;
+        List<PName> physicalNames = Collections.emptyList();
+        if (tableType == PTableType.VIEW) {
+            viewType = ViewType.fromSerializedValue(table.getViewType().toByteArray()[0]);
+        }
+        if(table.hasViewStatement()){
+            viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
+        }
+        if (tableType == PTableType.VIEW || viewIndexId != null) {
+            physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
+            for(int i = 0; i < table.getPhysicalNamesCount(); i++) {
+                physicalNames.add(PNameFactory.newName(table.getPhysicalNames(i).toByteArray()));
+            }
+        }
+        int baseColumnCount = -1;
+        if (table.hasBaseColumnCount()) {
+            baseColumnCount = table.getBaseColumnCount();
         }
-      }
-      
-      int baseColumnCount = -1;
-      if (table.hasBaseColumnCount()) {
-          baseColumnCount = table.getBaseColumnCount();
-      }
 
-      boolean rowKeyOrderOptimizable = false;
-      if (table.hasRowKeyOrderOptimizable()) {
-          rowKeyOrderOptimizable = table.getRowKeyOrderOptimizable();
-      }
-      long updateCacheFrequency = 0;
-      if (table.hasUpdateCacheFrequency()) {
-          updateCacheFrequency = table.getUpdateCacheFrequency();
-      }
-      boolean isNamespaceMapped=false;
-      if (table.hasIsNamespaceMapped()) {
-          isNamespaceMapped = table.getIsNamespaceMapped();
-      }
-      String autoParititonSeqName = null;
-      if (table.hasAutoParititonSeqName()) {
-          autoParititonSeqName = table.getAutoParititonSeqName();
-      }
-      boolean isAppendOnlySchema = false;
-      if (table.hasIsAppendOnlySchema()) {
-          isAppendOnlySchema = table.getIsAppendOnlySchema();
-      }
-      
-      try {
-        PTableImpl result = new PTableImpl();
-        result.init(tenantId, schemaName, tableName, tableType, indexState, timeStamp, sequenceNumber, pkName,
-            (bucketNum == NO_SALTING) ? null : bucketNum, columns, parentSchemaName, parentTableName, indexes,
-            isImmutableRows, physicalNames, defaultFamilyName, viewStatement, disableWAL,
-            multiTenant, storeNulls, viewType, viewIndexId, indexType, baseColumnCount, rowKeyOrderOptimizable,
-            isTransactional, updateCacheFrequency, indexDisableTimestamp, isNamespaceMapped, autoParititonSeqName, isAppendOnlySchema);
-        return result;
-      } catch (SQLException e) {
-        throw new RuntimeException(e); // Impossible
-      }
+        boolean rowKeyOrderOptimizable = false;
+        if (table.hasRowKeyOrderOptimizable()) {
+            rowKeyOrderOptimizable = table.getRowKeyOrderOptimizable();
+        }
+        long updateCacheFrequency = 0;
+        if (table.hasUpdateCacheFrequency()) {
+            updateCacheFrequency = table.getUpdateCacheFrequency();
+        }
+        boolean isNamespaceMapped=false;
+        if (table.hasIsNamespaceMapped()) {
+            isNamespaceMapped = table.getIsNamespaceMapped();
+        }
+        String autoParititonSeqName = null;
+        if (table.hasAutoParititonSeqName()) {
+            autoParititonSeqName = table.getAutoParititonSeqName();
+        }
+        boolean isAppendOnlySchema = false;
+        if (table.hasIsAppendOnlySchema()) {
+            isAppendOnlySchema = table.getIsAppendOnlySchema();
+        }
+        ImmutableStorageScheme storageScheme = null;
+        if (table.hasStorageScheme()) {
+            storageScheme = ImmutableStorageScheme.fromSerializedValue(table.getStorageScheme().toByteArray()[0]);
+        }
+        QualifierEncodingScheme qualifierEncodingScheme = null;
+        if (table.hasEncodingScheme()) {
+            qualifierEncodingScheme = QualifierEncodingScheme.fromSerializedValue(table.getEncodingScheme().toByteArray()[0]);
+        }
+        EncodedCQCounter encodedColumnQualifierCounter = null;
+        if ((!EncodedColumnsUtil.usesEncodedColumnNames(qualifierEncodingScheme) || tableType == PTableType.VIEW)) {
+        	encodedColumnQualifierCounter = PTable.EncodedCQCounter.NULL_COUNTER;
+        }
+        else {
+        	encodedColumnQualifierCounter = new EncodedCQCounter();
+        	if (table.getEncodedCQCountersList() != null) {
+        		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, qualifierEncodingScheme, encodedColumnQualifierCounter);
+            return result;
+        } catch (SQLException e) {
+            throw new RuntimeException(e); // Impossible
+        }
     }
 
     public static PTableProtos.PTable toProto(PTable table) {
@@ -1269,10 +1410,25 @@ 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.getImmutableStorageScheme() != null) {
+          builder.setStorageScheme(ByteStringer.wrap(new byte[]{table.getImmutableStorageScheme().getSerializedMetadataValue()}));
+      }
+      if (table.getEncodedCQCounter() != null) {
+          Map<String, Integer> values = table.getEncodedCQCounter().values();
+          for (Entry<String, Integer> cqCounter : values.entrySet()) {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.Builder cqBuilder = org.apache.phoenix.coprocessor.generated.PTableProtos.EncodedCQCounter.newBuilder();
+              cqBuilder.setColFamily(cqCounter.getKey());
+              cqBuilder.setCounter(cqCounter.getValue());
+              builder.addEncodedCQCounters(cqBuilder.build());
+          }
+      }
+      if (table.getEncodingScheme() != null) {
+          builder.setEncodingScheme(ByteStringer.wrap(new byte[]{table.getEncodingScheme().getSerializedMetadataValue()}));
+      }
       return builder.build();
     }
 
@@ -1342,4 +1498,54 @@ public class PTableImpl implements PTable {
         } else if (!key.equals(other.getKey())) return false;
         return true;
     }
+    
+    @Override
+    public ImmutableStorageScheme getImmutableStorageScheme() {
+        return immutableStorageScheme;
+    }
+    
+    @Override
+    public EncodedCQCounter getEncodedCQCounter() {
+        return encodedCQCounter;
+    }
+
+    @Override
+    public QualifierEncodingScheme getEncodingScheme() {
+        return qualifierEncodingScheme;
+    }
+    
+    private static final class KVColumnFamilyQualifier {
+        @Nonnull
+        private final String colFamilyName;
+        @Nonnull
+        private final byte[] colQualifier;
+
+        public KVColumnFamilyQualifier(String colFamilyName, byte[] colQualifier) {
+            Preconditions.checkArgument(colFamilyName != null && colQualifier != null,
+                "None of the arguments, column family name or column qualifier can be null");
+            this.colFamilyName = colFamilyName;
+            this.colQualifier = colQualifier;
+        }
+
+        @Override
+        public int hashCode() {
+            final int prime = 31;
+            int result = 1;
+            result = prime * result + colFamilyName.hashCode();
+            result = prime * result + Arrays.hashCode(colQualifier);
+            return result;
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (this == obj) return true;
+            if (obj == null) return false;
+            if (getClass() != obj.getClass()) return false;
+            KVColumnFamilyQualifier other = (KVColumnFamilyQualifier) obj;
+            if (!colFamilyName.equals(other.colFamilyName)) return false;
+            if (!Arrays.equals(colQualifier, other.colQualifier)) return false;
+            return true;
+        }
+
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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..d875982 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
@@ -24,14 +24,16 @@ public class ProjectedColumn extends DelegateColumn {
     private final int position;
     private final boolean nullable;
     private final ColumnRef sourceColumnRef;
+    private final byte[] cq;
 
-    public ProjectedColumn(PName name, PName familyName, int position, boolean nullable, ColumnRef sourceColumnRef) {
+    public ProjectedColumn(PName name, PName familyName, int position, boolean nullable, ColumnRef sourceColumnRef, byte[] cq) {
         super(sourceColumnRef.getColumn());
         this.name = name;
         this.familyName = familyName;
         this.position = position;
         this.nullable = nullable;
         this.sourceColumnRef = sourceColumnRef;
+        this.cq = cq;
     }
     
     @Override
@@ -39,6 +41,7 @@ public class ProjectedColumn extends DelegateColumn {
         return name;
     }
     
+    @Override
     public PName getFamilyName() {
         return familyName;
     }
@@ -52,7 +55,12 @@ public class ProjectedColumn extends DelegateColumn {
     public boolean isNullable() {
         return nullable;
     }
-
+    
+    @Override
+    public byte[] getColumnQualifierBytes() {
+        return cq;
+    }
+    
     public ColumnRef getSourceColumnRef() {
         return sourceColumnRef;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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 9962859..3282cc1 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
@@ -32,6 +32,7 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.util.SchemaUtil;
 
 public enum TableProperty {
@@ -145,6 +146,47 @@ public enum TableProperty {
         }       
 	    
 	},
+	
+	COLUMN_ENCODED_BYTES(PhoenixDatabaseMetaData.ENCODING_SCHEME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, false, false, false) {
+	    @Override
+        public Object getValue(Object value) {
+	        if (value instanceof String) {
+	            String strValue = (String) value;
+	            if ("NONE".equalsIgnoreCase(strValue)) {
+	                return (byte)0;
+	            } 
+	        } else {
+	            return value == null ? null : ((Number) value).byteValue();
+	        }
+	        return value;
+	    }
+
+		@Override
+		public Object getPTableValue(PTable table) {
+			return table.getEncodingScheme();
+		}	
+	    
+	},
+    
+    IMMUTABLE_STORAGE_SCHEME(PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME, COLUMN_FAMILY_NOT_ALLOWED_TABLE_PROPERTY, true, false, false) {
+        @Override
+        public ImmutableStorageScheme getValue(Object value) {
+            if (value == null) {
+                return null;
+            } else if (value instanceof String) {
+                String strValue = (String) value;
+                return ImmutableStorageScheme.valueOf(strValue.toUpperCase());
+            } else {
+                throw new IllegalArgumentException("Immutable storage scheme table property must be a string");
+            }
+        }
+
+        @Override
+        public Object getPTableValue(PTable table) {
+            return table.getImmutableStorageScheme();
+        }   
+        
+    }
     ;
 	
 	private final String propertyName;

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


[11/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..32e9f68 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
@@ -16,6 +16,7 @@
  * limitations under the License.
  */
 package org.apache.phoenix.compile;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
 
 import java.sql.SQLException;
 import java.util.ArrayList;
@@ -24,7 +25,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 +43,13 @@ 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.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;
 
@@ -120,7 +122,7 @@ public class TupleProjectionCompiler {
             PColumn sourceColumn = table.getPKColumns().get(i);
             ColumnRef sourceColumnRef = new ColumnRef(tableRef, sourceColumn.getPosition());
             PColumn column = new ProjectedColumn(sourceColumn.getName(), sourceColumn.getFamilyName(), 
-                    position++, sourceColumn.isNullable(), sourceColumnRef);
+                    position++, sourceColumn.isNullable(), sourceColumnRef, null);
             projectedColumns.add(column);
         }
         for (PColumn sourceColumn : table.getColumns()) {
@@ -132,18 +134,18 @@ public class TupleProjectionCompiler {
                     && !families.contains(sourceColumn.getFamilyName().getString()))
                 continue;
             PColumn column = new ProjectedColumn(sourceColumn.getName(), sourceColumn.getFamilyName(), 
-                    position++, sourceColumn.isNullable(), sourceColumnRef);
+                    position++, sourceColumn.isNullable(), sourceColumnRef, sourceColumn.getColumnQualifierBytes());
             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
         for (LocalIndexDataColumnRef sourceColumnRef : visitor.localIndexColumnRefSet) {
             PColumn column = new ProjectedColumn(sourceColumnRef.getColumn().getName(), 
                     sourceColumnRef.getColumn().getFamilyName(), position++, 
-                    sourceColumnRef.getColumn().isNullable(), sourceColumnRef);
+                    sourceColumnRef.getColumn().isNullable(), sourceColumnRef, sourceColumnRef.getColumn().getColumnQualifierBytes());
             projectedColumns.add(column);
         }
         
@@ -154,9 +156,9 @@ public class TupleProjectionCompiler {
                 null, null, table.isWALDisabled(), table.isMultiTenant(), table.getStoreNulls(), table.getViewType(),
                 table.getViewIndexId(),
                 table.getIndexType(), table.rowKeyOrderOptimizable(), table.isTransactional(), table.getUpdateCacheFrequency(), 
-                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema());
+                table.getIndexDisableTimestamp(), table.isNamespaceMapped(), table.getAutoPartitionSeqName(), table.isAppendOnlySchema(), table.getImmutableStorageScheme(), table.getEncodingScheme(), table.getEncodedCQCounter());
     }
-
+    
     public static PTable createProjectedTable(TableRef tableRef, List<ColumnRef> sourceColumnRefs, boolean retainPKColumns) throws SQLException {
         PTable table = tableRef.getTable();
         boolean hasSaltingColumn = retainPKColumns && table.getBucketNum() != null;
@@ -169,20 +171,23 @@ public class TupleProjectionCompiler {
             String aliasedName = tableRef.getTableAlias() == null ? 
                       SchemaUtil.getColumnName(table.getName().getString(), colName) 
                     : SchemaUtil.getColumnName(tableRef.getTableAlias(), colName);
-
-            PColumn column = new ProjectedColumn(PNameFactory.newName(aliasedName), 
-                    retainPKColumns && SchemaUtil.isPKColumn(sourceColumn) ? 
-                            null : PNameFactory.newName(TupleProjector.VALUE_COLUMN_FAMILY), 
-                    position++, sourceColumn.isNullable(), sourceColumnRef);
+            PName familyName =  SchemaUtil.isPKColumn(sourceColumn) ? (retainPKColumns ? null : PNameFactory.newName(VALUE_COLUMN_FAMILY)) : sourceColumn.getFamilyName();
+            PColumn column = new ProjectedColumn(PNameFactory.newName(aliasedName), familyName, 
+                    position++, sourceColumn.isNullable(), sourceColumnRef, sourceColumn.getColumnQualifierBytes());
             projectedColumns.add(column);
         }
+        EncodedCQCounter cqCounter = EncodedCQCounter.NULL_COUNTER;
+        if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
+            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(), table.getImmutableStorageScheme(), table.getEncodingScheme(), cqCounter);
     }
 
     // For extracting column references from single select statement

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..e5e18e3 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,8 @@ import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -84,11 +86,12 @@ public class UnionCompiler {
         for (int i = 0; i < plan.getProjector().getColumnCount(); i++) {
             ColumnProjector colProj = plan.getProjector().getColumnProjector(i);
             String name = selectNodes == null ? colProj.getName() : selectNodes.get(i).getAlias();
+            PName colName = PNameFactory.newName(name);
             PColumnImpl projectedColumn = new PColumnImpl(PNameFactory.newName(name),
                 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, colName.getBytes());
             projectedColumns.add(projectedColumn);
         }
         Long scn = statement.getConnection().getSCN();
@@ -98,7 +101,7 @@ public class UnionCompiler {
             null, null, projectedColumns, null, null, null, true, null, null, null, true,
             true, true, null, null, null, false, false, 0, 0L,
             SchemaUtil.isNamespaceMappingEnabled(PTableType.SUBQUERY,
-                statement.getConnection().getQueryServices().getProps()), null, false);
+                statement.getConnection().getQueryServices().getProps()), null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
         TableRef tableRef = new TableRef(null, tempTable, 0, false);
         return tableRef;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
index 18070d4..7a285a9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/UpsertCompiler.java
@@ -748,7 +748,7 @@ public class UpsertCompiler {
                             if (ptr.getLength() > 0) {
                                 byte[] uuidValue = ServerCacheClient.generateId();
                                 scan.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                scan.setAttribute(PhoenixIndexCodec.INDEX_MD, ptr.get());
+                                scan.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
                                 scan.setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
                             }
                             ResultIterator iterator = aggPlan.iterator();
@@ -917,10 +917,10 @@ public class UpsertCompiler {
                 UpdateColumnCompiler compiler = new UpdateColumnCompiler(context);
                 int nColumns = onDupKeyPairs.size();
                 List<Expression> updateExpressions = Lists.newArrayListWithExpectedSize(nColumns);
-                LinkedHashSet<PColumn>updateColumns = Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
+                LinkedHashSet<PColumn> updateColumns = Sets.newLinkedHashSetWithExpectedSize(nColumns + 1);
                 updateColumns.add(new PColumnImpl(
                         table.getPKColumns().get(0).getName(), // Use first PK column name as we know it won't conflict with others
-                        null, PVarbinary.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false));
+                        null, PVarbinary.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null, false, null, false, false, null));
                 for (Pair<ColumnName,ParseNode> columnPair : onDupKeyPairs) {
                     ColumnName colName = columnPair.getFirst();
                     PColumn updateColumn = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 39451b8..ed6c6cc 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
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.util.EncodedColumnsUtil.isPossibleToUseEncodedCQFilter;
+
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.util.Collections;
@@ -36,6 +38,7 @@ import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.visitor.KeyValueExpressionVisitor;
 import org.apache.phoenix.filter.MultiCFCQKeyValueComparisonFilter;
 import org.apache.phoenix.filter.MultiCQKeyValueComparisonFilter;
+import org.apache.phoenix.filter.MultiEncodedCQKeyValueComparisonFilter;
 import org.apache.phoenix.filter.RowKeyComparisonFilter;
 import org.apache.phoenix.filter.SingleCFCQKeyValueComparisonFilter;
 import org.apache.phoenix.filter.SingleCQKeyValueComparisonFilter;
@@ -46,17 +49,21 @@ import org.apache.phoenix.parse.ParseNodeFactory;
 import org.apache.phoenix.parse.SelectStatement;
 import org.apache.phoenix.parse.StatelessTraverseAllParseNodeVisitor;
 import org.apache.phoenix.parse.SubqueryParseNode;
+import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.AmbiguousColumnException;
 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.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 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.ExpressionUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.SchemaUtil;
 
@@ -169,12 +176,14 @@ public class WhereCompiler {
         public Expression visit(ColumnParseNode node) throws SQLException {
             ColumnRef ref = resolveColumn(node);
             TableRef tableRef = ref.getTableRef();
+            Expression newColumnExpression = ref.newColumnExpression(node.isTableNameCaseSensitive(), node.isCaseSensitive());
             if (tableRef.equals(context.getCurrentTable()) && !SchemaUtil.isPKColumn(ref.getColumn())) {
+                byte[] cq = tableRef.getTable().getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS 
+                		? QueryConstants.SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES : ref.getColumn().getColumnQualifierBytes();
                 // 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
@@ -195,7 +204,7 @@ public class WhereCompiler {
             // just use that.
             try {
                 if (!SchemaUtil.isPKColumn(ref.getColumn())) {
-                    table.getColumn(ref.getColumn().getName().getString());
+                    table.getColumnForColumnName(ref.getColumn().getName().getString());
                 }
             } catch (AmbiguousColumnException e) {
                 disambiguateWithFamily = true;
@@ -223,6 +232,7 @@ public class WhereCompiler {
 
             }
         }
+        
         public Count getCount() {
             return count;
         }
@@ -258,6 +268,8 @@ public class WhereCompiler {
                     return null;
                 }
             });
+            QualifierEncodingScheme encodingScheme = context.getCurrentTable().getTable().getEncodingScheme();
+            ImmutableStorageScheme storageScheme = context.getCurrentTable().getTable().getImmutableStorageScheme();
             switch (counter.getCount()) {
             case NONE:
                 PTable table = context.getResolver().getTables().get(0).getTable();
@@ -270,7 +282,9 @@ public class WhereCompiler {
                 filter = disambiguateWithFamily ? new SingleCFCQKeyValueComparisonFilter(whereClause) : new SingleCQKeyValueComparisonFilter(whereClause);
                 break;
             case MULTIPLE:
-                filter = disambiguateWithFamily ? new MultiCFCQKeyValueComparisonFilter(whereClause) : new MultiCQKeyValueComparisonFilter(whereClause);
+                filter = isPossibleToUseEncodedCQFilter(encodingScheme, storageScheme) ? new MultiEncodedCQKeyValueComparisonFilter(
+                        whereClause, encodingScheme) : (disambiguateWithFamily ? new MultiCFCQKeyValueComparisonFilter(
+                        whereClause) : new MultiCQKeyValueComparisonFilter(whereClause));
                 break;
             }
             scan.setFilter(filter);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 f6bd512..b4bda98 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
@@ -19,6 +19,7 @@ package org.apache.phoenix.coprocessor;
 
 import java.io.IOException;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
@@ -50,11 +51,15 @@ import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.KeyValueSchema;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 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.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
@@ -78,12 +83,19 @@ 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";
+    /* 
+    * Attribute to denote that the index maintainer has been serialized using its proto-buf presentation.
+    * Needed for backward compatibility purposes. TODO: get rid of this in next major release.
+    */
+    public static final String LOCAL_INDEX_BUILD_PROTO = "_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";
@@ -102,6 +114,12 @@ 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";
+    public final static String USE_NEW_VALUE_COLUMN_QUALIFIER = "_UseNewValueColumnQualifier";
+    public final static String QUALIFIER_ENCODING_SCHEME = "_QualifierEncodingScheme";
+    public final static String IMMUTABLE_STORAGE_ENCODING_SCHEME = "_ImmutableStorageEncodingScheme";
+    public final static String USE_ENCODED_COLUMN_QUALIFIER_LIST = "_UseEncodedColumnQualifierList";
     
     /**
      * Attribute name used to pass custom annotations in Scans and Mutations (later). Custom annotations
@@ -112,6 +130,8 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
     /** Exposed for testing */
     public static final String SCANNER_OPENED_TRACE_INFO = "Scanner opened on server";
     protected Configuration rawConf;
+    protected QualifierEncodingScheme encodingScheme;
+    protected boolean useNewValueColumnQualifier;
 
     @Override
     public void start(CoprocessorEnvironment e) throws IOException {
@@ -184,6 +204,8 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
             // start exclusive and the stop inclusive.
             ScanUtil.setupReverseScan(scan);
         }
+        this.encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
+        this.useNewValueColumnQualifier = EncodedColumnsUtil.useNewValueColumnQualifier(scan);
         return s;
     }
 
@@ -308,14 +330,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 Region 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);
     }
 
     /**
@@ -333,7 +355,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,
@@ -341,7 +363,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();
@@ -438,11 +460,13 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                             tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
                     }
                     if (projector != null) {
-                        Tuple tuple = projector.projectResults(new ResultTuple(Result.create(result)));
+                        Tuple toProject = useQualifierAsListIndex ? new PositionBasedResultTuple(result) : new ResultTuple(Result.create(result));
+                        Tuple tuple = projector.projectResults(toProject, useNewValueColumnQualifier);
                         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;
@@ -472,6 +496,16 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                         if (result.isEmpty()) {
                             return next;
                         }
+                        IndexUtil.wrapResultUsingOffset(c, result, offset, dataColumns,
+                            tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
+                    }
+                    if (projector != null) {
+                        Tuple toProject = useQualifierAsListIndex ? new PositionBasedMultiKeyValueTuple(result) : new ResultTuple(Result.create(result));
+                        Tuple tuple = projector.projectResults(toProject, useNewValueColumnQualifier);
+                        result.clear();
+                        result.add(tuple.getValue(0));
+                        if(arrayElementCell != null)
+                            result.add(arrayElementCell);
                     }
                     IndexUtil.wrapResultUsingOffset(c, result, offset, dataColumns,
                         tupleProjector, dataRegion, indexMaintainer, viewConstants, ptr);
@@ -524,7 +558,7 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
 
             private int replaceArrayIndexElement(final Set<KeyValueColumnExpression> arrayKVRefs,
                     final Expression[] arrayFuncRefs, List<Cell> result) {
-                // make a copy of the results array here, as we're modifying it below
+             // make a copy of the results array here, as we're modifying it below
                 MultiKeyValueTuple tuple = new MultiKeyValueTuple(ImmutableList.copyOf(result));
                 // The size of both the arrays would be same?
                 // Using KeyValueSchema to set and retrieve the value
@@ -532,14 +566,15 @@ abstract public class BaseScannerRegionObserver extends BaseRegionObserver {
                 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);
+                        ListIterator<Cell> itr = result.listIterator();
+                        while (itr.hasNext()) {
+                            Cell kv = itr.next();
                             if (Bytes.equals(kvExp.getColumnFamily(), 0, kvExp.getColumnFamily().length,
                                     kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength())
-                                && Bytes.equals(kvExp.getColumnName(), 0, kvExp.getColumnName().length,
+                                && Bytes.equals(kvExp.getColumnQualifier(), 0, kvExp.getColumnQualifier().length,
                                         kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength())) {
                                 // remove the kv that has the full array values.
-                                result.remove(idx);
+                                itr.remove();
                                 break;
                             }
                         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 2c194c9..67cc114 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
@@ -46,6 +46,7 @@ import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.regionserver.Region;
 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,9 +63,13 @@ 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.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.LogUtil;
@@ -123,15 +128,20 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
                         .getEnvironment().getConfiguration());
 
         RegionScanner innerScanner = s;
-
-        byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
-        List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
+        boolean useProto = false;
+        byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD_PROTO);
+        useProto = localIndexBytes != null;
+        if (localIndexBytes == null) {
+            localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
+        }
+        List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes, useProto);
         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 = EncodedColumnsUtil.useQualifierAsIndex(EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan));
         if (ScanUtil.isLocalIndex(scan) || (j == null && p != null)) {
             if (dataColumns != null) {
                 tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
@@ -139,14 +149,14 @@ 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);
-        }
+                    getWrappedScanner(c, innerScanner, offset, scan, dataColumns, tupleProjector, 
+                            c.getEnvironment().getRegion(), indexMaintainers == null ? null : indexMaintainers.get(0), viewConstants, p, tempPtr, useQualifierAsIndex);
+        } 
 
         if (j != null) {
             innerScanner =
                     new HashJoinRegionScanner(innerScanner, p, j, ScanUtil.getTenantId(scan),
-                            c.getEnvironment());
+                            c.getEnvironment(), useQualifierAsIndex, useNewValueColumnQualifier);
         }
 
         long limit = Long.MAX_VALUE;
@@ -380,7 +390,9 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             estDistVals = Math.max(MIN_DISTINCT_VALUES,
                             (int) (Bytes.toInt(estDistValsBytes) * 1.5f));
         }
-
+        
+        Pair<Integer, Integer> minMaxQualifiers = EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan);
+        boolean useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan));
         final boolean spillableEnabled =
                 conf.getBoolean(GROUPBY_SPILLABLE_ATTRIB, DEFAULT_GROUPBY_SPILLABLE);
 
@@ -391,12 +403,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)));
             }
-
             Region region = c.getEnvironment().getRegion();
             boolean acquiredLock = false;
             try {
@@ -404,7 +414,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(), encodingScheme) : 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
@@ -439,7 +449,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.
@@ -454,6 +464,8 @@ public class GroupedAggregateRegionObserver extends BaseScannerRegionObserver {
             logger.debug(LogUtil.addCustomAnnotations("Grouped aggregation over ordered rows with scan " + scan + ", group by "
                     + expressions + ", aggregators " + aggregators, ScanUtil.getCustomAnnotations(scan)));
         }
+        final Pair<Integer, Integer> minMaxQualifiers = EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan);
+        final boolean useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(minMaxQualifiers);
         return new BaseRegionScanner(scanner) {
             private long rowCount = 0;
             private ImmutableBytesPtr currentKey = null;
@@ -463,7 +475,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
@@ -476,7 +488,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(), encodingScheme) : new ArrayList<Cell>();
                             // Results are potentially returned even when the return
                             // value of s.next is false
                             // since this is an indication of whether or not there

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 4340886..79ff4b0 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
@@ -43,6 +43,7 @@ import org.apache.phoenix.parse.JoinTableNode.JoinType;
 import org.apache.phoenix.schema.IllegalDataException;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.ValueBitSet;
+import org.apache.phoenix.schema.tuple.PositionBasedResultTuple;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ServerUtil;
@@ -62,9 +63,11 @@ public class HashJoinRegionScanner implements RegionScanner {
     private List<Tuple>[] tempTuples;
     private ValueBitSet tempDestBitSet;
     private ValueBitSet[] tempSrcBitSet;
-
+    private final boolean useQualifierAsListIndex;
+    private final boolean useNewValueColumnQualifier;
+    
     @SuppressWarnings("unchecked")
-    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesPtr tenantId, RegionCoprocessorEnvironment env) throws IOException {
+    public HashJoinRegionScanner(RegionScanner scanner, TupleProjector projector, HashJoinInfo joinInfo, ImmutableBytesPtr tenantId, RegionCoprocessorEnvironment env, boolean useQualifierAsIndex, boolean useNewValueColumnQualifier) throws IOException {
         this.env = env;
         this.scanner = scanner;
         this.projector = projector;
@@ -105,17 +108,18 @@ public class HashJoinRegionScanner implements RegionScanner {
             this.tempDestBitSet = ValueBitSet.newInstance(joinInfo.getJoinedSchema());
             this.projector.setValueBitSet(tempDestBitSet);
         }
+        this.useQualifierAsListIndex = useQualifierAsIndex;
+        this.useNewValueColumnQualifier = useNewValueColumnQualifier;
     }
 
     private void processResults(List<Cell> result, boolean hasBatchLimit) throws IOException {
         if (result.isEmpty())
             return;
-
-        Tuple tuple = new ResultTuple(Result.create(result));
+        Tuple tuple = useQualifierAsListIndex ? new PositionBasedResultTuple(result) : new ResultTuple(Result.create(result));
         // For backward compatibility. In new versions, HashJoinInfo.forceProjection()
         // always returns true.
         if (joinInfo.forceProjection()) {
-            tuple = projector.projectResults(tuple);
+            tuple = projector.projectResults(tuple, useNewValueColumnQualifier);
         }
 
         // TODO: fix below Scanner.next() and Scanner.nextRaw() methods as well.
@@ -148,7 +152,7 @@ public class HashJoinRegionScanner implements RegionScanner {
             } else {
                 KeyValueSchema schema = joinInfo.getJoinedSchema();
                 if (!joinInfo.forceProjection()) { // backward compatibility
-                    tuple = projector.projectResults(tuple);
+                    tuple = projector.projectResults(tuple, useNewValueColumnQualifier);
                 }
                 resultQueue.offer(tuple);
                 for (int i = 0; i < count; i++) {
@@ -175,8 +179,8 @@ public class HashJoinRegionScanner implements RegionScanner {
                             Tuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
                                     lhs : TupleProjector.mergeProjectedValue(
                                             (ProjectedValueTuple) lhs, schema, tempDestBitSet,
-                                            null, joinInfo.getSchemas()[i], tempSrcBitSet[i],
-                                            joinInfo.getFieldPositions()[i]);
+                                            null, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                            joinInfo.getFieldPositions()[i], useNewValueColumnQualifier);
                             resultQueue.offer(joined);
                             continue;
                         }
@@ -184,8 +188,8 @@ public class HashJoinRegionScanner implements RegionScanner {
                             Tuple joined = tempSrcBitSet[i] == ValueBitSet.EMPTY_VALUE_BITSET ?
                                     lhs : TupleProjector.mergeProjectedValue(
                                             (ProjectedValueTuple) lhs, schema, tempDestBitSet,
-                                            t, joinInfo.getSchemas()[i], tempSrcBitSet[i],
-                                            joinInfo.getFieldPositions()[i]);
+                                            t, joinInfo.getSchemas()[i], tempSrcBitSet[i], 
+                                            joinInfo.getFieldPositions()[i], useNewValueColumnQualifier);
                             resultQueue.offer(joined);
                         }
                     }
@@ -317,7 +321,6 @@ public class HashJoinRegionScanner implements RegionScanner {
                 processResults(result, false); // TODO detect if limit used here
                 result.clear();
             }
-            
             return nextInQueue(result);
         } catch (Throwable t) {
             ServerUtil.throwIOException(env.getRegion().getRegionInfo().getRegionNameAsString(), t);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 76bda44..2005c94 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,8 @@ 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_BYTES;
+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 +36,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.ENCODING_SCHEME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES;
@@ -57,6 +60,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;
@@ -89,7 +93,6 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -189,8 +192,11 @@ 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.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
@@ -209,10 +215,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;
@@ -283,6 +291,8 @@ 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 KeyValue ENCODING_SCHEME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ENCODING_SCHEME_BYTES);
     
     private static final List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
             EMPTY_KEYVALUE_KV,
@@ -309,7 +319,9 @@ 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,
+            ENCODING_SCHEME_KV
             );
     static {
         Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
@@ -339,6 +351,8 @@ 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);
+    private static final int QUALIFIER_ENCODING_SCHEME_INDEX = TABLE_KV_COLUMNS.indexOf(ENCODING_SCHEME_KV);
 
     // KeyValues for Column
     private static final KeyValue DECIMAL_DIGITS_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
@@ -352,6 +366,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue IS_VIEW_REFERENCED_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_VIEW_REFERENCED_BYTES);
     private static final KeyValue COLUMN_DEF_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_DEF_BYTES);
     private static final KeyValue IS_ROW_TIMESTAMP_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_ROW_TIMESTAMP_BYTES);
+    private static final KeyValue COLUMN_QUALIFIER_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_QUALIFIER_BYTES);
+
     private static final List<KeyValue> COLUMN_KV_COLUMNS = Arrays.<KeyValue>asList(
             DECIMAL_DIGITS_KV,
             COLUMN_SIZE_KV,
@@ -364,11 +380,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,
+            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);
@@ -380,6 +398,7 @@ 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 COLUMN_QUALIFIER_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_QUALIFIER_KV);
     
     private static final int LINK_TYPE_INDEX = 0;
 
@@ -717,8 +736,16 @@ 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);
+        
+        boolean isPkColumn = famName == null || famName.getString() == null;
+        Cell columnQualifierKV = colKeyValues[COLUMN_QUALIFIER_INDEX];
+        // Older tables won't have column qualifier metadata present. To make things simpler, just set the
+        // column qualifier bytes by using the column name.
+        byte[] columnQualifierBytes = columnQualifierKV != null ? 
+                Arrays.copyOfRange(columnQualifierKV.getValueArray(),
+                    columnQualifierKV.getValueOffset(), columnQualifierKV.getValueOffset()
+                            + columnQualifierKV.getValueLength()) : (isPkColumn ? null : colName.getBytes());
+        PColumn column = new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable, position-1, sortOrder, arraySize, viewConstant, isViewReferenced, expressionStr, isRowTimestamp, false, columnQualifierBytes);
         columns.add(column);
     }
     
@@ -926,37 +953,55 @@ 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
+        ImmutableStorageScheme storageScheme = storageSchemeKv == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ImmutableStorageScheme
+                .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
+                        storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength()));
+        Cell encodingSchemeKv = tableKeyValues[QUALIFIER_ENCODING_SCHEME_INDEX];
+        QualifierEncodingScheme encodingScheme = encodingSchemeKv == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : QualifierEncodingScheme
+                .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
+                    encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
         
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();
         List<PName> physicalTables = Lists.newArrayList();
         PName parentTableName = tableType == INDEX ? dataTableName : null;
         PName parentSchemaName = tableType == INDEX ? schemaName : null;
+        EncodedCQCounter cqCounter =
+                (!EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme) || tableType == PTableType.VIEW) ? PTable.EncodedCQCounter.NULL_COUNTER
+                        : new EncodedCQCounter();
         while (true) {
-          results.clear();
-          scanner.next(results);
-          if (results.isEmpty()) {
-              break;
-          }
-          Cell colKv = results.get(LINK_TYPE_INDEX);
-          int colKeyLength = colKv.getRowLength();
-          PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength-offset);
-          int colKeyOffset = offset + colName.getBytes().length + 1;
-          PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength-colKeyOffset);
-          if (colName.getString().isEmpty() && famName != null) {
-              LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
-              if (linkType == LinkType.INDEX_TABLE) {
-                  addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes);
-              } else if (linkType == LinkType.PHYSICAL_TABLE) {
-                  physicalTables.add(famName);
-              } else if (linkType == LinkType.PARENT_TABLE) {
-                  parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
-                  parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
-              }
-          } else {
-              addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null);
-          }
+            results.clear();
+            scanner.next(results);
+            if (results.isEmpty()) {
+                break;
+            }
+            Cell colKv = results.get(LINK_TYPE_INDEX);
+            if (colKv != null) {
+                int colKeyLength = colKv.getRowLength();
+                PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength-offset);
+                int colKeyOffset = offset + colName.getBytes().length + 1;
+                PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength-colKeyOffset);
+                if (isQualifierCounterKV(colKv)) {
+                    Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
+                    cqCounter.setValue(famName.getString(), value);
+                } else {
+                    if (colName.getString().isEmpty() && famName != null) {
+                        LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
+                        if (linkType == LinkType.INDEX_TABLE) {
+                            addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes);
+                        } else if (linkType == LinkType.PHYSICAL_TABLE) {
+                            physicalTables.add(famName);
+                        } else if (linkType == LinkType.PARENT_TABLE) {
+                            parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
+                            parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
+                        }
+                    } else {
+                        addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null);
+                    }
+                } 
+            }
         }
         // Avoid querying the stats table because we're holding the rowLock here. Issuing an RPC to a remote
         // server while holding this lock is a bad idea and likely to cause contention.
@@ -964,9 +1009,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, encodingScheme, 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);
@@ -1437,7 +1490,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // tableMetadata and set the view statement and partition column correctly
                 if (parentTable!=null && parentTable.getAutoPartitionSeqName()!=null) {
                     long autoPartitionNum = 1;
-                    final Properties props = new Properties();
                     try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
                         Statement stmt = connection.createStatement()) {
                         String seqName = parentTable.getAutoPartitionSeqName();
@@ -1505,46 +1557,46 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 Short indexId = null;
                 if (request.hasAllocateIndexId() && request.getAllocateIndexId()) {
                     String tenantIdStr = tenantIdBytes.length == 0 ? null : Bytes.toString(tenantIdBytes);
-                    try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)){
-                    PName physicalName = parentTable.getPhysicalName();
-                    int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
-                    SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
+                    try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class)) {
+                        PName physicalName = parentTable.getPhysicalName();
+                        int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
+                        SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
                             nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
                         // TODO Review Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
                         // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
                         // or not. 
-                    long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
-                    try {
-                        connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
+                        long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
+                        try {
+                            connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
                                 Short.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
-                    } catch (SequenceAlreadyExistsException e) {
-                    }
-                    long[] seqValues = new long[1];
-                    SQLException[] sqlExceptions = new SQLException[1];
-                    connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
+                        } catch (SequenceAlreadyExistsException e) {
+                        }
+                        long[] seqValues = new long[1];
+                        SQLException[] sqlExceptions = new SQLException[1];
+                        connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
                             HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
-                    if (sqlExceptions[0] != null) {
-                        throw sqlExceptions[0];
-                    }
-                    long seqValue = seqValues[0];
-                    if (seqValue > Short.MAX_VALUE) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        done.run(builder.build());
-                        return;
-                    }
-                    Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
-                    NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
-                    List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
-                    Cell cell = cells.get(0);
-                    PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
-                    Object val = dataType.toObject(seqValue, PLong.INSTANCE);
-                    byte[] bytes = new byte [dataType.getByteSize() + 1];
-                    dataType.toBytes(val, bytes, 0);
-                    Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
+                        if (sqlExceptions[0] != null) {
+                            throw sqlExceptions[0];
+                        }
+                        long seqValue = seqValues[0];
+                        if (seqValue > Short.MAX_VALUE) {
+                            builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
+                            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                            done.run(builder.build());
+                            return;
+                        }
+                        Put tableHeaderPut = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetadata);
+                        NavigableMap<byte[], List<Cell>> familyCellMap = tableHeaderPut.getFamilyCellMap();
+                        List<Cell> cells = familyCellMap.get(TABLE_FAMILY_BYTES);
+                        Cell cell = cells.get(0);
+                        PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
+                        Object val = dataType.toObject(seqValue, PLong.INSTANCE);
+                        byte[] bytes = new byte [dataType.getByteSize() + 1];
+                        dataType.toBytes(val, bytes, 0);
+                        Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
                             cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
-                    cells.add(indexIdCell);
-                    indexId = (short) seqValue;
+                        cells.add(indexIdCell);
+                        indexId = (short) seqValue;
                     }
                 }
                 
@@ -2169,6 +2221,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 int pkCount = getVarChars(m.getRow(), rkmd);
                 // check if this put is for adding a column
                 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)));
@@ -2221,8 +2274,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.getColumnForColumnName(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) {
@@ -2551,8 +2604,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.getColumnForColumnName(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.
@@ -2618,7 +2671,25 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     
     private MetaDataMutationResult validateColumnForAddToBaseTable(PColumn existingViewColumn, Put columnToBeAdded, PTable basePhysicalTable, boolean isColumnToBeAddPkCol, PTable view) {
         if (existingViewColumn != null) {
-            
+            if (EncodedColumnsUtil.usesEncodedColumnNames(basePhysicalTable) && !SchemaUtil.isPKColumn(existingViewColumn)) {
+                /*
+                 * If the column already exists in a view, then we cannot add the column to the base
+                 * table. The reason is subtle and is as follows: consider the case where a table
+                 * has two views where both the views have the same key value column KV. Now, we
+                 * dole out encoded column qualifiers for key value columns in views by using the
+                 * counters stored in the base physical table. So the KV column can have different
+                 * column qualifiers for the two views. For example, 11 for VIEW1 and 12 for VIEW2.
+                 * This naturally extends to rows being inserted using the two views having
+                 * different column qualifiers for the column named KV. Now, when an attempt is made
+                 * to add column KV to the base table, we cannot decide which column qualifier
+                 * should that column be assigned. It cannot be a number different than 11 or 12
+                 * since a query like SELECT KV FROM BASETABLE would return null for KV which is
+                 * incorrect since column KV is present in rows inserted from the two views. We
+                 * cannot use 11 or 12 either because we will then incorrectly return value of KV
+                 * column inserted using only one view.
+                 */
+                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
+            }
             // Validate data type is same
             int baseColumnDataType = getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES);
             if (baseColumnDataType != existingViewColumn.getDataType().getSqlType()) {
@@ -2848,6 +2919,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     return mutationResult;
                             } 
                         }
+                    } else if (type == PTableType.VIEW
+                            && EncodedColumnsUtil.usesEncodedColumnNames(table)) {
+                        /*
+                         * When adding a column to a view that uses encoded column name scheme, we
+                         * need to modify the CQ counters stored in the view's physical table. So to
+                         * make sure clients get the latest PTable, we need to invalidate the cache
+                         * entry.
+                         */
+                        invalidateList.add(new ImmutableBytesPtr(MetaDataUtil
+                                .getPhysicalTableRowForView(table)));
                     }
                     for (Mutation m : tableMetaData) {
                         byte[] key = m.getRow();
@@ -2861,7 +2942,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;
@@ -3114,7 +3195,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;
@@ -3203,10 +3284,12 @@ 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());
+            ColumnReference colDropRef = new ColumnReference(columnToDelete.getFamilyName().getBytes(), columnToDelete.getColumnQualifierBytes());
+            boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(columnToDeleteInfo);
+            boolean isCoveredColumn = indexMaintainer.getCoveredColumns().contains(colDropRef);
             // 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.
@@ -3227,9 +3310,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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 83290db..dd445ce 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -85,8 +85,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0 = MIN_TABLE_TIMESTAMP + 18;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_8_1 = MIN_TABLE_TIMESTAMP + 18;
     public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0 = MIN_TABLE_TIMESTAMP + 20;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 = MIN_TABLE_TIMESTAMP + 25;
     // MIN_SYSTEM_TABLE_TIMESTAMP needs to be set to the max of all the MIN_SYSTEM_TABLE_TIMESTAMP_* constants
-    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0;
+    public static final long MIN_SYSTEM_TABLE_TIMESTAMP = MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0;
     
     // ALWAYS update this map whenever rolling out a new release (major, minor or patch release). 
     // Key is the SYSTEM.CATALOG timestamp for the version and value is the version string.
@@ -101,6 +102,7 @@ public abstract class MetaDataProtocol extends MetaDataService {
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_7_0, "4.7.x");
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_8_0, "4.8.x");
         TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0, "4.9.x");
+        TIMESTAMP_VERSION_MAP.put(MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0, "4.10.x");
     }
     
     public static final String CURRENT_CLIENT_VERSION = PHOENIX_MAJOR_VERSION + "." + PHOENIX_MINOR_VERSION + "." + PHOENIX_PATCH_NUMBER; 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index 8f36803..9482d37 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -387,7 +387,7 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
 							IndexMaintainer.serializeAdditional(dataPTable, indexMetaDataPtr, indexesToPartiallyRebuild,
 									conn);
 							byte[] attribValue = ByteUtil.copyKeyBytesIfNecessary(indexMetaDataPtr);
-							dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+							dataTableScan.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
                             LOG.info("Starting to partially build indexes:" + indexesToPartiallyRebuild
                                     + " on data table:" + dataPTable.getName() + " with the earliest disable timestamp:"
                                     + earliestDisableTimestamp + " till "
@@ -511,4 +511,4 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
 				put);
 
 	}
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 ade88db..02b05f9 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
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.coprocessor;
 
+import static org.apache.phoenix.util.EncodedColumnsUtil.getMinMaxQualifiersFromScan;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -44,6 +46,7 @@ import org.apache.phoenix.execute.TupleProjector;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.OrderByExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
 import org.apache.phoenix.expression.function.ArrayIndexFunction;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
@@ -57,10 +60,13 @@ import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.KeyValueSchema;
 import org.apache.phoenix.schema.KeyValueSchema.KeyValueSchemaBuilder;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.ValueBitSet;
 import org.apache.phoenix.schema.tuple.ResultTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.ScanUtil;
 import org.apache.phoenix.util.ServerUtil;
@@ -107,7 +113,7 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         }
     }
 
-    public static OrderedResultIterator deserializeFromScan(Scan scan, RegionScanner s) {
+    private static OrderedResultIterator deserializeFromScan(Scan scan, RegionScanner s) {
         byte[] topN = scan.getAttribute(BaseScannerRegionObserver.TOPN);
         if (topN == null) {
             return null;
@@ -125,7 +131,8 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
                 orderByExpression.readFields(input);
                 orderByExpressions.add(orderByExpression);
             }
-            ResultIterator inner = new RegionScannerResultIterator(s);
+            QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
+            ResultIterator inner = new RegionScannerResultIterator(s, EncodedColumnsUtil.getMinMaxQualifiersFromScan(scan), encodingScheme);
             return new OrderedResultIterator(inner, orderByExpressions, thresholdBytes, limit >= 0 ? limit : null, null,
                     estimatedRowSize);
         } catch (IOException e) {
@@ -151,7 +158,9 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
             DataInputStream input = new DataInputStream(stream);
             int arrayKVRefSize = WritableUtils.readVInt(input);
             for (int i = 0; i < arrayKVRefSize; i++) {
-                KeyValueColumnExpression kvExp = new KeyValueColumnExpression();
+                ImmutableStorageScheme scheme = EncodedColumnsUtil.getImmutableStorageScheme(scan);
+                KeyValueColumnExpression kvExp = scheme != ImmutableStorageScheme.ONE_CELL_PER_COLUMN ? new SingleCellColumnExpression()
+                        : new KeyValueColumnExpression();
                 kvExp.readFields(input);
                 arrayKVRefs.add(kvExp);
             }
@@ -209,8 +218,13 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
         if (dataColumns != null) {
             tupleProjector = IndexUtil.getTupleProjector(scan, dataColumns);
             dataRegion = c.getEnvironment().getRegion();
-            byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
-            List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes);
+            boolean useProto = false;
+            byte[] localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD_PROTO);
+            useProto = localIndexBytes != null;
+            if (localIndexBytes == null) {
+                localIndexBytes = scan.getAttribute(LOCAL_INDEX_BUILD);
+            }
+            List<IndexMaintainer> indexMaintainers = localIndexBytes == null ? null : IndexMaintainer.deserialize(localIndexBytes, useProto);
             indexMaintainer = indexMaintainers.get(0);
             viewConstants = IndexUtil.deserializeViewConstantsFromScan(scan);
             byte[] txState = scan.getAttribute(BaseScannerRegionObserver.TX_STATE);
@@ -219,21 +233,22 @@ public class ScanRegionObserver extends BaseScannerRegionObserver {
 
         final TupleProjector p = TupleProjector.deserializeProjectorFromScan(scan);
         final HashJoinInfo j = HashJoinInfo.deserializeHashJoinFromScan(scan);
+        boolean useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(getMinMaxQualifiersFromScan(scan)) && scan.getAttribute(BaseScannerRegionObserver.TOPN) != null;
         innerScanner =
                 getWrappedScanner(c, innerScanner, arrayKVRefs, arrayFuncRefs, offset, scan,
                     dataColumns, tupleProjector, dataRegion, indexMaintainer, tx,
-                    viewConstants, kvSchema, kvSchemaBitSet, j == null ? p : null, ptr);
+                    viewConstants, kvSchema, kvSchemaBitSet, j == null ? p : null, ptr, useQualifierAsIndex);
 
         final ImmutableBytesPtr tenantId = ScanUtil.getTenantId(scan);
         if (j != null) {
-            innerScanner = new HashJoinRegionScanner(innerScanner, p, j, tenantId, c.getEnvironment());
+            innerScanner = new HashJoinRegionScanner(innerScanner, p, j, tenantId, c.getEnvironment(), useQualifierAsIndex, useNewValueColumnQualifier);
         }
         if (scanOffset != null) {
             innerScanner = getOffsetScanner(c, innerScanner,
-                    new OffsetResultIterator(new RegionScannerResultIterator(innerScanner), scanOffset),
+                    new OffsetResultIterator(new RegionScannerResultIterator(innerScanner, getMinMaxQualifiersFromScan(scan), encodingScheme), scanOffset),
                     scan.getAttribute(QueryConstants.LAST_SCAN) != null);
         }
-        final OrderedResultIterator iterator = deserializeFromScan(scan,innerScanner);
+        final OrderedResultIterator iterator = deserializeFromScan(scan, innerScanner);
         if (iterator == null) {
             return innerScanner;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
index bf889d5..98f57ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ServerCachingEndpointImpl.java
@@ -72,7 +72,7 @@ public class ServerCachingEndpointImpl extends ServerCachingService implements C
           (Class<ServerCacheFactory>) Class.forName(request.getCacheFactory().getClassName());
       ServerCacheFactory cacheFactory = serverCacheFactoryClass.newInstance();
       tenantCache.addServerCache(new ImmutableBytesPtr(request.getCacheId().toByteArray()),
-        cachePtr, txState, cacheFactory);
+        cachePtr, txState, cacheFactory, request.hasHasProtoBufIndexMaintainer() && request.getHasProtoBufIndexMaintainer());
     } catch (Throwable e) {
       ProtobufUtil.setControllerException(controller, new IOException(e));
     }


[03/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..5a5b355
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/EncodedColumnQualiferCellsList.java
@@ -0,0 +1,581 @@
+/*
+ * 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.Collection;
+import java.util.ConcurrentModificationException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.NoSuchElementException;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import org.apache.hadoop.hbase.Cell;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+
+/**
+ * List implementation that provides indexed based look up when the cell column qualifiers are positive numbers. 
+ * These qualifiers are generated by using one of the column qualifier encoding schemes specified in {@link ImmutableStorageScheme}. 
+ * The api methods in this list assume that the caller wants to see
+ * and add only non null elements in the list. 
+ * <p>
+ * Please note that this implementation doesn't implement all the optional methods of the 
+ * {@link List} interface. Such unsupported methods could violate the basic invariance of the list that every cell with
+ * an encoded column qualifier has a fixed position in the list.
+ * </p>
+ * <p>
+ * An important performance characteristic of this list is that doing look up on the basis of index via {@link #get(int)}
+ * is an O(n) operation. This makes iterating through the list using {@link #get(int)} an O(n^2) operation.
+ * Instead, for iterating through the list, one should use the iterators created through {@link #iterator()} or 
+ * {@link #listIterator()}. Do note that getting an element using {@link #getCellForColumnQualifier(int)} is an O(1) operation
+ * and should generally be the way for accessing elements in the list.
+ * </p> 
+ */
+@NotThreadSafe
+public class EncodedColumnQualiferCellsList implements List<Cell> {
+
+    private int minQualifier;
+    private int maxQualifier;
+    private int nonReservedRangeOffset;
+    private final Cell[] array;
+    private int numNonNullElements;
+    private int firstNonNullElementIdx = -1;
+    private static final int RESERVED_RANGE_SIZE = ENCODED_CQ_COUNTER_INITIAL_VALUE - ENCODED_EMPTY_COLUMN_NAME;
+    // Used by iterators to figure out if the list was structurally modified.
+    private int modCount = 0;
+    private final QualifierEncodingScheme encodingScheme;
+
+    public EncodedColumnQualiferCellsList(int minQ, int maxQ, QualifierEncodingScheme encodingScheme) {
+        checkArgument(minQ <= maxQ, "Invalid arguments. Min: " + minQ
+                + ". Max: " + maxQ);
+        this.minQualifier = minQ;
+        this.maxQualifier = maxQ;
+        int size = 0;
+        if (maxQ < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            size = RESERVED_RANGE_SIZE;
+        } else if (minQ < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            size = (maxQ - minQ + 1);
+        } else {
+            size = RESERVED_RANGE_SIZE + (maxQ - minQ + 1);
+        }
+        this.array = new Cell[size];
+        this.nonReservedRangeOffset = minQ > ENCODED_CQ_COUNTER_INITIAL_VALUE ? minQ  - ENCODED_CQ_COUNTER_INITIAL_VALUE : 0;
+        this.encodingScheme = encodingScheme;
+    }
+
+    @Override
+    public int size() {
+        return numNonNullElements;
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return numNonNullElements == 0;
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return indexOf(o) >= 0;
+    }
+
+    @Override
+    public Object[] toArray() {
+        Object[] toReturn = new Object[numNonNullElements];
+        int counter = 0;
+        if (numNonNullElements > 0) {
+            for (int i = 0; i < array.length; i++) {
+                if (array[i] != null) {
+                    toReturn[counter++] = array[i];
+                }
+            }
+        }
+        return toReturn;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public <T> T[] toArray(T[] a) {
+        T[] toReturn =
+                (T[]) java.lang.reflect.Array.newInstance(a.getClass().getComponentType(),
+                    numNonNullElements);
+        int counter = 0;
+        for (int i = 0; i < array.length; i++) {
+            if (array[i] != null) {
+                toReturn[counter++] = (T) array[i];
+            }
+        }
+        return toReturn;
+    }
+
+    @Override
+    public boolean add(Cell e) {
+        if (e == null) {
+            throw new NullPointerException();
+        }
+        int columnQualifier = encodingScheme.decode(e.getQualifierArray(), e.getQualifierOffset(), e.getQualifierLength());
+                
+        checkQualifierRange(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        if (array[idx] == null) {
+            numNonNullElements++;
+        }
+        array[idx] = e;
+        if (firstNonNullElementIdx == -1) {
+            firstNonNullElementIdx = idx;
+        } else if (idx < firstNonNullElementIdx) {
+            firstNonNullElementIdx = idx;
+        }
+        modCount++;
+        /*
+         * Note that we don't care about equality of the element being added with the element
+         * already present at the index.
+         */
+        return true;
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        if (o == null) {
+            return false;
+        }
+        Cell e = (Cell) o;
+        int i = 0;
+        while (i < array.length) {
+            if (array[i] != null && array[i].equals(e)) {
+                array[i] = null;
+                numNonNullElements--;
+                if (numNonNullElements == 0) {
+                    firstNonNullElementIdx = -1;
+                } else if (firstNonNullElementIdx == i) {
+                    // the element being removed was the first non-null element we knew
+                    while (i < array.length && (array[i]) == null) {
+                        i++;
+                    }
+                    if (i < array.length) {
+                        firstNonNullElementIdx = i;
+                    } else {
+                        firstNonNullElementIdx = -1;
+                    }
+                }
+                modCount++;
+                return true;
+            }
+            i++;
+        }
+        return false;
+    }
+
+    @Override
+    public boolean containsAll(Collection<?> c) {
+        boolean containsAll = true;
+        Iterator<?> itr = c.iterator();
+        while (itr.hasNext()) {
+            containsAll &= (indexOf(itr.next()) >= 0);
+        }
+        return containsAll;
+    }
+
+    @Override
+    public boolean addAll(Collection<? extends Cell> c) {
+        boolean changed = false;
+        for (Cell cell : c) {
+            if (c == null) {
+                throw new NullPointerException();
+            }
+            changed |= add(cell);
+        }
+        return changed;
+    }
+
+    @Override
+    public boolean addAll(int index, Collection<? extends Cell> c) {
+        throwGenericUnsupportedOperationException();
+        return false;
+    }
+
+    @Override
+    public boolean removeAll(Collection<?> c) {
+        Iterator<?> itr = c.iterator();
+        boolean changed = false;
+        while (itr.hasNext()) {
+            changed |= remove(itr.next());
+        }
+        return changed;
+    }
+
+    @Override
+    public boolean retainAll(Collection<?> collection) {
+        boolean changed = false;
+        // Optimize if the passed collection is an instance of EncodedColumnQualiferCellsList
+        if (collection instanceof EncodedColumnQualiferCellsList) {
+            EncodedColumnQualiferCellsList list = (EncodedColumnQualiferCellsList) collection;
+            ListIterator<Cell> listItr = this.listIterator();
+            while (listItr.hasNext()) {
+                Cell cellInThis = listItr.next();
+                int qualifier = encodingScheme.decode(cellInThis.getQualifierArray(),
+                            cellInThis.getQualifierOffset(), cellInThis.getQualifierLength());
+                try {
+                    Cell cellInParam = list.getCellForColumnQualifier(qualifier);
+                    if (cellInParam != null && cellInParam.equals(cellInThis)) {
+                        continue;
+                    }
+                    listItr.remove();
+                    changed = true;
+                } catch (IndexOutOfBoundsException expected) {
+                    // this could happen when the qualifier of cellInParam lies out of
+                    // the range of this list.
+                    listItr.remove();
+                    changed = true;
+                }
+            }
+        } else {
+            throw new UnsupportedOperationException(
+                    "Operation only supported for collections of type EncodedColumnQualiferCellsList");
+        }
+        return changed;
+    }
+
+    @Override
+    public void clear() {
+        for (int i = 0; i < array.length; i++) {
+            array[i] = null;
+        }
+        firstNonNullElementIdx = -1;
+        numNonNullElements = 0;
+        modCount++;
+    }
+
+    @Override
+    public Cell get(int index) {
+        rangeCheck(index);
+        int numNonNullElementsFound = 0;
+        for (int i = firstNonNullElementIdx; i < array.length; i++) {
+            if (array[i] != null) {
+                numNonNullElementsFound++;
+                if (numNonNullElementsFound == index + 1) {
+                    return array[i];
+                }
+            }
+        }
+        throw new IllegalStateException("There was no element present in the list at index "
+                + index + " even though number of elements in the list are " + size());
+    }
+
+    @Override
+    public Cell set(int index, Cell e) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public void add(int index, Cell element) {
+        throwGenericUnsupportedOperationException();
+    }
+
+    @Override
+    public Cell remove(int index) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public int indexOf(Object o) {
+        if (o == null || isEmpty()) {
+            return -1;
+        } else {
+            int numNonNull = -1;
+            for (int i = 0; i < array.length; i++) {
+                if (array[i] != null) {
+                    numNonNull++;
+                }
+                if (o.equals(array[i])) {
+                    return numNonNull;
+                }
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public int lastIndexOf(Object o) {
+        if (o == null || isEmpty()) {
+            return -1;
+        }
+        int lastIndex = numNonNullElements;
+        for (int i = array.length - 1; i >= 0; i--) {
+            if (array[i] != null) {
+                lastIndex--;
+            }
+            if (o.equals(array[i])) {
+                return lastIndex;
+            }
+        }
+        return -1;
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator() {
+        return new ListItr();
+    }
+
+    @Override
+    public ListIterator<Cell> listIterator(int index) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public List<Cell> subList(int fromIndex, int toIndex) {
+        throwGenericUnsupportedOperationException();
+        return null;
+    }
+
+    @Override
+    public Iterator<Cell> iterator() {
+        return new Itr();
+    }
+
+    public Cell getCellForColumnQualifier(byte[] qualifierBytes) {
+        int columnQualifier = encodingScheme.decode(qualifierBytes);
+        return getCellForColumnQualifier(columnQualifier);
+    }
+    
+    public Cell getCellForColumnQualifier(byte[] qualifierBytes, int offset, int length) {
+        int columnQualifier = encodingScheme.decode(qualifierBytes, offset, length);
+        return getCellForColumnQualifier(columnQualifier);
+    }
+    
+    private Cell getCellForColumnQualifier(int columnQualifier) {
+        checkQualifierRange(columnQualifier);
+        int idx = getArrayIndex(columnQualifier);
+        Cell c = array[idx];
+        return c;
+    }
+
+    public Cell getFirstCell() {
+        if (firstNonNullElementIdx == -1) {
+            throw new NoSuchElementException("No elements present in the list");
+        }
+        return array[firstNonNullElementIdx];
+    }
+
+    private void checkQualifierRange(int qualifier) {
+        if (qualifier < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            return; // space in the array for reserved range is always allocated. 
+        }
+        if (qualifier < minQualifier || qualifier > maxQualifier) {
+            throw new IndexOutOfBoundsException("Qualifier " + qualifier
+                    + " is out of the valid range - (" + minQualifier + ", " + maxQualifier + ")");
+        }
+    }
+
+    private void rangeCheck(int index) {
+        if (index < 0 || index >= size()) {
+            throw new IndexOutOfBoundsException();
+        }
+    }
+
+    private int getArrayIndex(int columnQualifier) {
+        checkArgument(columnQualifier >= ENCODED_EMPTY_COLUMN_NAME);
+        if (columnQualifier < ENCODED_CQ_COUNTER_INITIAL_VALUE) {
+            return columnQualifier;
+        }
+        return columnQualifier - nonReservedRangeOffset;
+    }
+
+    private void throwGenericUnsupportedOperationException() {
+        throw new UnsupportedOperationException(
+                "Operation cannot be supported because it potentially violates the invariance contract of this list implementation");
+    }
+
+    private class Itr implements Iterator<Cell> {
+        protected int nextIndex = 0;
+        protected int lastRet = -1;
+        protected int expectedModCount = modCount;
+        
+        private Itr() {
+            moveForward(true);
+        }
+
+        @Override
+        public boolean hasNext() {
+            return nextIndex != -1;
+        }
+
+        @Override
+        public Cell next() {
+            checkForCoModification();
+            if (!hasNext()) {
+                throw new NoSuchElementException();
+            }
+            Cell next = array[nextIndex];
+            lastRet = nextIndex;
+            moveForward(false);
+            modCount++;
+            expectedModCount = modCount;
+            return next;
+        }
+
+        @Override
+        public void remove() {
+            if (lastRet < 0) {
+                throw new IllegalStateException();
+            }
+            checkForCoModification();
+            array[lastRet] = null;
+            lastRet = -1;
+            numNonNullElements--;
+            modCount++;
+            expectedModCount = modCount;
+        }
+
+        protected void moveForward(boolean init) {
+            int i = init ? 0 : nextIndex + 1;
+            while (i < array.length && (array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                nextIndex = i;
+            } else {
+                nextIndex = -1;
+            }
+        }
+        
+        protected void checkForCoModification() {
+            if (modCount != expectedModCount) {
+                throw new ConcurrentModificationException();
+            }
+        }
+
+    }
+
+    private class ListItr extends Itr implements ListIterator<Cell> {
+        private int previousIndex = -1;
+        
+        private ListItr() {
+            moveForward(true);
+        }
+
+        @Override
+        public boolean hasNext() {
+            return nextIndex != -1;
+        }
+
+        @Override
+        public boolean hasPrevious() {
+            return previousIndex != -1;
+        }
+
+        @Override
+        public Cell previous() {
+            if (previousIndex == -1) {
+                throw new NoSuchElementException();
+            }
+            checkForCoModification();
+            lastRet = previousIndex;
+            movePointersBackward();
+            return array[lastRet];
+        }
+
+        @Override
+        public int nextIndex() {
+            return nextIndex;
+        }
+
+        @Override
+        public int previousIndex() {
+            return previousIndex;
+        }
+
+        @Override
+        public void remove() {
+            if (lastRet == nextIndex) {
+                moveNextPointer(nextIndex);
+            }
+            super.remove();
+            expectedModCount = modCount;
+        }
+
+        @Override
+        public void set(Cell e) {
+            if (lastRet == -1) {
+                throw new IllegalStateException();
+            }
+            int columnQualifier = encodingScheme.decode(e.getQualifierArray(), e.getQualifierOffset(), e.getQualifierLength());                    
+            int idx = getArrayIndex(columnQualifier);
+            if (idx != lastRet) {
+                throw new IllegalArgumentException("Cell " + e + " with column qualifier "
+                        + columnQualifier + " belongs at index " + idx
+                        + ". It cannot be added at the position " + lastRet
+                        + " to which the previous next() or previous() was pointing to.");
+            }
+            EncodedColumnQualiferCellsList.this.add(e);
+            expectedModCount = modCount;
+        }
+
+        @Override
+        public void add(Cell e) {
+            throwGenericUnsupportedOperationException();
+        }
+        
+        @Override
+        protected void moveForward(boolean init) {
+            if (!init) {
+                previousIndex = nextIndex;
+            }
+            int i = init ? 0 : nextIndex + 1; 
+            moveNextPointer(i);
+        }
+
+        private void moveNextPointer(int i) {
+            while (i < array.length && (array[i]) == null) {
+                i++;
+            }
+            if (i < array.length) {
+                nextIndex = i;
+            } else {
+                nextIndex = -1;
+            }
+        }
+
+        private void movePointersBackward() {
+            nextIndex = previousIndex;
+            int i = previousIndex - 1;
+            movePreviousPointer(i);
+        }
+
+        private void movePreviousPointer(int i) {
+            for (; i >= 0; i--) {
+                if (array[i] != null) {
+                    previousIndex = i;
+                    break;
+                }
+            }
+            if (i < 0) {
+                previousIndex = -1;
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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..34097e9
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/tuple/PositionBasedMultiKeyValueTuple.java
@@ -0,0 +1,90 @@
+/*
+ * 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;
+
+/**
+ * Tuple that is closely tied with {@link EncodedColumnQualiferCellsList}. It essentially provides a
+ * way of getting hold of cell belonging to a cq/cf by doing a position based look up as opposed to
+ * a MultiKeyValueTuple where we have to do a binary search in the List.
+ */
+public class PositionBasedMultiKeyValueTuple extends BaseTuple {
+    private EncodedColumnQualiferCellsList values;
+
+    public PositionBasedMultiKeyValueTuple() {
+    }
+
+    public PositionBasedMultiKeyValueTuple(List<Cell> values) {
+        checkArgument(values instanceof EncodedColumnQualiferCellsList,
+            "PositionBasedMultiKeyValueTuple only works with lists of type EncodedColumnQualiferCellsList");
+        this.values = (EncodedColumnQualiferCellsList) values;
+    }
+
+    /** Caller must not modify the list that is passed here */
+    @Override
+    public void setKeyValues(List<Cell> values) {
+        checkArgument(values instanceof EncodedColumnQualiferCellsList,
+            "PositionBasedMultiKeyValueTuple only works with lists of type EncodedColumnQualiferCellsList");
+        this.values = (EncodedColumnQualiferCellsList) values;
+    }
+
+    @Override
+    public void getKey(ImmutableBytesWritable ptr) {
+        Cell value = values.getFirstCell();
+        ptr.set(value.getRowArray(), value.getRowOffset(), value.getRowLength());
+    }
+
+    @Override
+    public boolean isImmutable() {
+        return true;
+    }
+
+    @Override
+    public Cell getValue(byte[] family, byte[] qualifier) {
+        return values.getCellForColumnQualifier(qualifier);
+    }
+
+    @Override
+    public String toString() {
+        return values.toString();
+    }
+
+    @Override
+    public int size() {
+        return values.size();
+    }
+
+    @Override
+    public Cell getValue(int index) {
+        return values.get(index);
+    }
+
+    @Override
+    public boolean getValue(byte[] family, byte[] qualifier, ImmutableBytesWritable ptr) {
+        Cell kv = getValue(family, qualifier);
+        if (kv == null) return false;
+        ptr.set(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength());
+        return true;
+    }
+}

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

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
index 1d2cfb2..f31f272 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataType.java
@@ -22,19 +22,15 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.sql.Types;
 import java.text.Format;
-import java.util.LinkedList;
-import java.util.List;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ConstraintViolationException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.ValueSchema;
-import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TrustedByteArrayOutputStream;
@@ -74,8 +70,11 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
           this, actualModifer, desiredModifier, true);
     }
 
-    public static final byte ARRAY_SERIALIZATION_VERSION = 1;
-
+    // array serialization format where bytes can be used as part of the row key
+    public static final byte SORTABLE_SERIALIZATION_VERSION = 1;
+    // array serialization format where bytes are immutable (does not support prepend/append or sorting)
+    public static final byte IMMUTABLE_SERIALIZATION_VERSION = 2;
+    
     protected PArrayDataType(String sqlTypeName, int sqlType, Class clazz, PDataCodec codec, int ordinal) {
         super(sqlTypeName, sqlType, clazz, codec, ordinal);
     }
@@ -186,9 +185,17 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         oStream.write(sepByte);
     }
 
-    public static boolean useShortForOffsetArray(int maxOffset) {
-        // If the max offset is less than Short.MAX_VALUE then offset array can use short
-        if (maxOffset <= (2 * Short.MAX_VALUE)) { return true; }
+    // this method is only for append/prepend/concat operations which are only supported for the SORTABLE_SERIALIZATION_VERSION
+    public static boolean useShortForOffsetArray(int maxoffset) {
+    	return useShortForOffsetArray(maxoffset, SORTABLE_SERIALIZATION_VERSION);
+    }
+    
+    public static boolean useShortForOffsetArray(int maxoffset, byte serializationVersion) {
+    	if (serializationVersion == IMMUTABLE_SERIALIZATION_VERSION) {
+    		 return (maxoffset <= Short.MAX_VALUE && maxoffset >= Short.MIN_VALUE );
+    	}
+    	// If the max offset is less than Short.MAX_VALUE then offset array can use short
+    	else if (maxoffset <= (2 * Short.MAX_VALUE)) { return true; }
         // else offset array can use Int
         return false;
     }
@@ -342,126 +349,20 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         return createPhoenixArray(bytes, offset, length, sortOrder, baseType, maxLength, desiredDataType);
     }
 
-    public static boolean positionAtArrayElement(Tuple tuple, ImmutableBytesWritable ptr, int index,
-            Expression arrayExpr, PDataType pDataType, Integer maxLen) {
-        if (!arrayExpr.evaluate(tuple, ptr)) {
-            return false;
-        } else if (ptr.getLength() == 0) { return true; }
-
-        // Given a ptr to the entire array, set ptr to point to a particular element within that array
-        // given the type of an array element (see comments in PDataTypeForArray)
-        positionAtArrayElement(ptr, index - 1, pDataType, maxLen);
-        return true;
-    }
-
-    public static void positionAtArrayElement(ImmutableBytesWritable ptr, int arrayIndex, PDataType baseDataType,
-            Integer byteSize) {
-        byte[] bytes = ptr.get();
-        int initPos = ptr.getOffset();
-        if (!baseDataType.isFixedWidth()) {
-            int noOfElements = Bytes.toInt(bytes,
-                    (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)), Bytes.SIZEOF_INT);
-            boolean useShort = true;
-            if (noOfElements < 0) {
-                noOfElements = -noOfElements;
-                useShort = false;
-            }
-            if (arrayIndex >= noOfElements) {
-                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-                return;
-            }
-
-            int indexOffset = Bytes.toInt(bytes,
-                    (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT))) + ptr.getOffset();
-            if (arrayIndex >= noOfElements) {
-                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-            } else {
-                // Skip those many offsets as given in the arrayIndex
-                // If suppose there are 5 elements in the array and the arrayIndex = 3
-                // This means we need to read the 4th element of the array
-                // So inorder to know the length of the 4th element we will read the offset of 4th element and the
-                // offset of 5th element.
-                // Subtracting the offset of 5th element and 4th element will give the length of 4th element
-                // So we could just skip reading the other elements.
-                int currOffset = getOffset(bytes, arrayIndex, useShort, indexOffset);
-                int elementLength = 0;
-                if (arrayIndex == (noOfElements - 1)) {
-                    elementLength = (bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : indexOffset
-                            - (currOffset + initPos) - 3;
-                } else {
-                    elementLength = (bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : getOffset(bytes,
-                            arrayIndex + 1, useShort, indexOffset) - currOffset - 1;
-                }
-                ptr.set(bytes, currOffset + initPos, elementLength);
-            }
-        } else {
-            int elemByteSize = (byteSize == null ? baseDataType.getByteSize() : byteSize);
-            int offset = arrayIndex * elemByteSize;
-            if (offset >= ptr.getLength()) {
-                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
-            } else {
-                ptr.set(bytes, ptr.getOffset() + offset, elemByteSize);
-            }
-        }
-    }
-
-    public static void positionAtArrayElement(ImmutableBytesWritable ptr, int arrayIndex, PDataType baseDataType,
-            Integer byteSize, int offset, int length, int noOfElements, boolean first) {
-        byte[] bytes = ptr.get();
-        if (!baseDataType.isFixedWidth()) {
-            int indexOffset = Bytes.toInt(bytes, (offset + length - (Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT)))
-                    + offset;
-            boolean useShort = true;
-            if (first) {
-                int count = Bytes.toInt(bytes,
-                        (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)), Bytes.SIZEOF_INT);
-                if (count < 0) {
-                    count = -count;
-                    useShort = false;
-                }
-            }
-            if (arrayIndex >= noOfElements) {
-                return;
-            } else {
-                // Skip those many offsets as given in the arrayIndex
-                // If suppose there are 5 elements in the array and the arrayIndex = 3
-                // This means we need to read the 4th element of the array
-                // So inorder to know the length of the 4th element we will read the offset of 4th element and the
-                // offset of 5th element.
-                // Subtracting the offset of 5th element and 4th element will give the length of 4th element
-                // So we could just skip reading the other elements.
-                int currOffset = getOffset(bytes, arrayIndex, useShort, indexOffset);
-                int elementLength = 0;
-                if (arrayIndex == (noOfElements - 1)) {
-                    elementLength = (bytes[currOffset + offset] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + offset] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : indexOffset
-                            - (currOffset + offset) - 3;
-                } else {
-                    elementLength = (bytes[currOffset + offset] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + offset] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : getOffset(bytes,
-                            arrayIndex + 1, useShort, indexOffset) - currOffset - 1;
-                }
-                ptr.set(bytes, currOffset + offset, elementLength);
-            }
-        } else {
-            int elemByteSize = (byteSize == null ? baseDataType.getByteSize() : byteSize);
-            offset += arrayIndex * elemByteSize;
-            if (offset >= offset + length) {
-                return;
-            } else {
-                ptr.set(bytes, offset, elemByteSize);
-            }
-        }
+    static int getOffset(byte[] bytes, int arrayIndex, boolean useShort, int indexOffset, byte serializationVersion) {
+        return Math.abs(getSerializedOffset(bytes, arrayIndex, useShort, indexOffset, serializationVersion));
     }
 
-    private static int getOffset(byte[] bytes, int arrayIndex, boolean useShort, int indexOffset) {
-        int offset;
+	static int getSerializedOffset(byte[] bytes, int arrayIndex, boolean useShort, int indexOffset, byte serializationVersion) {
+		int offset;
         if (useShort) {
             offset = indexOffset + (Bytes.SIZEOF_SHORT * arrayIndex);
-            return Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT) + Short.MAX_VALUE;
+            return Bytes.toShort(bytes, offset, Bytes.SIZEOF_SHORT) + (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION ? 0 : Short.MAX_VALUE);
         } else {
             offset = indexOffset + (Bytes.SIZEOF_INT * arrayIndex);
             return Bytes.toInt(bytes, offset, Bytes.SIZEOF_INT);
         }
-    }
+	}
 
     private static int getOffset(ByteBuffer indexBuffer, int arrayIndex, boolean useShort, int indexOffset) {
         int offset;
@@ -484,58 +385,18 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
     }
 
     /**
-     * creates array bytes
+     * creates array bytes using the SORTABLE_SERIALIZATION_VERSION format
      * @param rowKeyOrderOptimizable TODO
      */
     private byte[] createArrayBytes(TrustedByteArrayOutputStream byteStream, DataOutputStream oStream,
             PhoenixArray array, int noOfElements, PDataType baseType, SortOrder sortOrder, boolean rowKeyOrderOptimizable) {
-        try {
-            if (!baseType.isFixedWidth()) {
-                int[] offsetPos = new int[noOfElements];
-                int nulls = 0;
-                for (int i = 0; i < noOfElements; i++) {
-                    byte[] bytes = array.toBytes(i);
-                    if (bytes.length == 0) {
-                        offsetPos[i] = byteStream.size();
-                        nulls++;
-                    } else {
-                        nulls = serializeNulls(oStream, nulls);
-                        offsetPos[i] = byteStream.size();
-                        if (sortOrder == SortOrder.DESC) {
-                            SortOrder.invert(bytes, 0, bytes, 0, bytes.length);
-                        }
-                        oStream.write(bytes, 0, bytes.length);
-                        oStream.write(getSeparatorByte(rowKeyOrderOptimizable, sortOrder));
-                    }
-                }
-                // Double seperator byte to show end of the non null array
-                writeEndSeperatorForVarLengthArray(oStream, sortOrder, rowKeyOrderOptimizable);
-                noOfElements = PArrayDataType.serailizeOffsetArrayIntoStream(oStream, byteStream, noOfElements,
-                        offsetPos[offsetPos.length - 1], offsetPos);
-                serializeHeaderInfoIntoStream(oStream, noOfElements);
-            } else {
-                for (int i = 0; i < noOfElements; i++) {
-                    byte[] bytes = array.toBytes(i);
-                    int length = bytes.length;
-                    if (sortOrder == SortOrder.DESC) {
-                        SortOrder.invert(bytes, 0, bytes, 0, bytes.length);
-                    }
-                    oStream.write(bytes, 0, length);
-                }
-            }
-            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-            ptr.set(byteStream.getBuffer(), 0, byteStream.size());
-            return ByteUtil.copyKeyBytesIfNecessary(ptr);
-        } catch (IOException e) {
-            try {
-                byteStream.close();
-                oStream.close();
-            } catch (IOException ioe) {
-
-            }
+        PArrayDataTypeEncoder builder =
+                new PArrayDataTypeEncoder(byteStream, oStream, noOfElements, baseType, sortOrder, rowKeyOrderOptimizable);
+        for (int i = 0; i < noOfElements; i++) {
+            byte[] bytes = array.toBytes(i);
+            builder.appendValue(bytes);
         }
-        // This should not happen
-        return null;
+        return builder.encode();
     }
 
     public static boolean appendItemToArray(ImmutableBytesWritable ptr, int length, int offset, byte[] arrayBytes,
@@ -557,7 +418,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
 
         byte[] newArray;
         if (!baseType.isFixedWidth()) {
-
+        	byte serializationVersion = arrayBytes[offset + length - Bytes.SIZEOF_BYTE];
             int offsetArrayPosition = Bytes.toInt(arrayBytes, offset + length - Bytes.SIZEOF_INT - Bytes.SIZEOF_INT
                     - Bytes.SIZEOF_BYTE, Bytes.SIZEOF_INT);
             int offsetArrayLength = length - offsetArrayPosition - Bytes.SIZEOF_INT - Bytes.SIZEOF_INT
@@ -612,7 +473,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                     int off = newOffsetArrayPosition;
                     for (int arrayIndex = 0; arrayIndex < Math.abs(arrayLength) - 1; arrayIndex++) {
                         Bytes.putInt(newArray, off,
-                                getOffset(arrayBytes, arrayIndex, true, offsetArrayPosition + offset));
+                                getOffset(arrayBytes, arrayIndex, true, offsetArrayPosition + offset, serializationVersion));
                         off += Bytes.SIZEOF_INT;
                     }
 
@@ -659,6 +520,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
 
         byte[] newArray;
         if (!baseType.isFixedWidth()) {
+        	byte serializationVersion = arrayBytes[offset + length - Bytes.SIZEOF_BYTE];
             int offsetArrayPosition = Bytes.toInt(arrayBytes, offset + length - Bytes.SIZEOF_INT - Bytes.SIZEOF_INT
                     - Bytes.SIZEOF_BYTE, Bytes.SIZEOF_INT);
             int offsetArrayLength = length - offsetArrayPosition - Bytes.SIZEOF_INT - Bytes.SIZEOF_INT
@@ -668,7 +530,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             // checks whether offset array consists of shorts or integers
             boolean useInt = offsetArrayLength / arrayLength == Bytes.SIZEOF_INT;
             boolean convertToInt = false;
-            int endElementPosition = getOffset(arrayBytes, arrayLength - 1, !useInt, offsetArrayPosition + offset)
+            int endElementPosition = getOffset(arrayBytes, arrayLength - 1, !useInt, offsetArrayPosition + offset, serializationVersion)
                     + elementLength + Bytes.SIZEOF_BYTE;
             int newOffsetArrayPosition;
             int lengthIncrease;
@@ -679,7 +541,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                 int nulls = 1;
                 // counts the number of nulls which are already at the beginning of the array
                 for (int index = 0; index < arrayLength; index++) {
-                    int currOffset = getOffset(arrayBytes, index, !useInt, offsetArrayPosition + offset);
+                    int currOffset = getOffset(arrayBytes, index, !useInt, offsetArrayPosition + offset, serializationVersion);
                     if (arrayBytes[offset + currOffset] == QueryConstants.SEPARATOR_BYTE) {
                         nulls++;
                     } else {
@@ -709,7 +571,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                 // ex: initial array - 0 45(inverted) 65 0 66 0 0 0 after prepending null - 0 46(inverted) 65 0 66 0 0 0
                 lengthIncrease = nRemainingNulls == 1 ? (nMultiplesOver255 == 0 ? 2 * Bytes.SIZEOF_BYTE
                         : Bytes.SIZEOF_BYTE) : 0;
-                endElementPosition = getOffset(arrayBytes, arrayLength - 1, !useInt, offsetArrayPosition + offset)
+                endElementPosition = getOffset(arrayBytes, arrayLength - 1, !useInt, offsetArrayPosition + offset, serializationVersion)
                         + lengthIncrease;
                 if (!useInt) {
                     if (PArrayDataType.useShortForOffsetArray(endElementPosition)) {
@@ -785,8 +647,9 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
 
         currentPosition += offsetArrayElementSize;
         boolean nullsAtBeginning = true;
+        byte serializationVersion = arrayBytes[offset + length - Bytes.SIZEOF_BYTE];
         for (int arrayIndex = 0; arrayIndex < arrayLength - 1; arrayIndex++) {
-            int oldOffset = getOffset(arrayBytes, arrayIndex, useShortPrevious, offsetArrayPosition + offset);
+            int oldOffset = getOffset(arrayBytes, arrayIndex, useShortPrevious, offsetArrayPosition + offset, serializationVersion);
             if (arrayBytes[offset + oldOffset] == QueryConstants.SEPARATOR_BYTE && nullsAtBeginning) {
                 if (useShortNew) {
                     Bytes.putShort(newArray, currentPosition, (short)(oldOffset - Short.MAX_VALUE));
@@ -820,6 +683,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         byte[] newArray;
 
         if (!baseType.isFixedWidth()) {
+        	byte serializationVersion1 = array1Bytes[array1BytesOffset + array1BytesLength - Bytes.SIZEOF_BYTE];
             int offsetArrayPositionArray1 = Bytes.toInt(array1Bytes, array1BytesOffset + array1BytesLength
                     - Bytes.SIZEOF_INT - Bytes.SIZEOF_INT - Bytes.SIZEOF_BYTE, Bytes.SIZEOF_INT);
             int offsetArrayPositionArray2 = Bytes.toInt(array2Bytes, array2BytesOffset + array2BytesLength
@@ -837,7 +701,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             boolean useIntNewArray = false;
             // count nulls at the end of array 1
             for (int index = actualLengthOfArray1 - 1; index > -1; index--) {
-                int offset = getOffset(array1Bytes, index, !useIntArray1, array1BytesOffset + offsetArrayPositionArray1);
+                int offset = getOffset(array1Bytes, index, !useIntArray1, array1BytesOffset + offsetArrayPositionArray1, serializationVersion1);
                 if (array1Bytes[array1BytesOffset + offset] == QueryConstants.SEPARATOR_BYTE || array1Bytes[array1BytesOffset + offset] == QueryConstants.DESC_SEPARATOR_BYTE) {
                     nullsAtTheEndOfArray1++;
                 } else {
@@ -847,8 +711,9 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             // count nulls at the beginning of the array 2
             int array2FirstNonNullElementOffset = 0;
             int array2FirstNonNullIndex = 0;
+            byte serializationVersion2 = array2Bytes[array2BytesOffset + array2BytesLength - Bytes.SIZEOF_BYTE];
             for (int index = 0; index < actualLengthOfArray2; index++) {
-                int offset = getOffset(array2Bytes, index, !useIntArray2, array2BytesOffset + offsetArrayPositionArray2);
+                int offset = getOffset(array2Bytes, index, !useIntArray2, array2BytesOffset + offsetArrayPositionArray2, serializationVersion2);
                 if (array2Bytes[array2BytesOffset + offset] == QueryConstants.SEPARATOR_BYTE) {
                     nullsAtTheBeginningOfArray2++;
                 } else {
@@ -870,7 +735,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             int newOffsetArrayPosition = offsetArrayPositionArray1 + offsetArrayPositionArray2 + lengthIncreaseForNulls
                     - 2 * Bytes.SIZEOF_BYTE;
             int endElementPositionOfArray2 = getOffset(array2Bytes, actualLengthOfArray2 - 1, !useIntArray2,
-                    array2BytesOffset + offsetArrayPositionArray2);
+                    array2BytesOffset + offsetArrayPositionArray2, serializationVersion2);
             int newEndElementPosition = lengthIncreaseForNulls + endElementPositionOfArray2 + offsetArrayPositionArray1
                     - 2 * Bytes.SIZEOF_BYTE;
             // Creates a byte array to store the concatenated array
@@ -902,14 +767,14 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                 // offsets for the elements from array 1. Simply copied.
                 for (int index = 0; index < actualLengthOfArray1; index++) {
                     int offset = getOffset(array1Bytes, index, !useIntArray1, array1BytesOffset
-                            + offsetArrayPositionArray1);
+                            + offsetArrayPositionArray1, serializationVersion1);
                     Bytes.putInt(newArray, currentPosition, offset);
                     currentPosition += Bytes.SIZEOF_INT;
                 }
                 // offsets for nulls in the middle
                 for (int index = 0; index < array2FirstNonNullIndex; index++) {
                     int offset = getOffset(array2Bytes, index, !useIntArray2, array2BytesOffset
-                            + offsetArrayPositionArray2);
+                            + offsetArrayPositionArray2, serializationVersion2);
                     Bytes.putInt(newArray, currentPosition, offset + array2StartingPosition);
                     currentPosition += Bytes.SIZEOF_INT;
                 }
@@ -918,7 +783,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                         + (bytesForNullsAfter == 0 ? 0 : Bytes.SIZEOF_BYTE);
                 for (int index = array2FirstNonNullIndex; index < actualLengthOfArray2; index++) {
                     int offset = getOffset(array2Bytes, index, !useIntArray2, array2BytesOffset
-                            + offsetArrayPositionArray2);
+                            + offsetArrayPositionArray2, serializationVersion2);
                     Bytes.putInt(newArray, currentPosition, offset - array2FirstNonNullElementOffset
                             + part2NonNullStartingPosition);
                     currentPosition += Bytes.SIZEOF_INT;
@@ -927,14 +792,14 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                 // offsets for the elements from array 1. Simply copied.
                 for (int index = 0; index < actualLengthOfArray1; index++) {
                     int offset = getOffset(array1Bytes, index, !useIntArray1, array1BytesOffset
-                            + offsetArrayPositionArray1);
+                            + offsetArrayPositionArray1, serializationVersion1);
                     Bytes.putShort(newArray, currentPosition, (short)(offset - Short.MAX_VALUE));
                     currentPosition += Bytes.SIZEOF_SHORT;
                 }
                 // offsets for nulls in the middle
                 for (int index = 0; index < array2FirstNonNullIndex; index++) {
                     int offset = getOffset(array2Bytes, index, !useIntArray2, array2BytesOffset
-                            + offsetArrayPositionArray2);
+                            + offsetArrayPositionArray2, serializationVersion2);
                     Bytes.putShort(newArray, currentPosition,
                             (short)(offset + array2StartingPosition - Short.MAX_VALUE));
                     currentPosition += Bytes.SIZEOF_SHORT;
@@ -944,7 +809,7 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
                         + (bytesForNullsAfter == 0 ? 0 : Bytes.SIZEOF_BYTE);
                 for (int index = array2FirstNonNullIndex; index < actualLengthOfArray2; index++) {
                     int offset = getOffset(array2Bytes, index, !useIntArray2, array2BytesOffset
-                            + offsetArrayPositionArray2);
+                            + offsetArrayPositionArray2, serializationVersion2);
                     Bytes.putShort(newArray, currentPosition, (short)(offset - array2FirstNonNullElementOffset
                             + part2NonNullStartingPosition - Short.MAX_VALUE));
                     currentPosition += Bytes.SIZEOF_SHORT;
@@ -1013,13 +878,13 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         ptr.set(PVarcharArray.INSTANCE.toBytes(phoenixArray, PVarchar.INSTANCE, sortOrder));
         return true;
     }
-
-    public static int serailizeOffsetArrayIntoStream(DataOutputStream oStream, TrustedByteArrayOutputStream byteStream,
-            int noOfElements, int maxOffset, int[] offsetPos) throws IOException {
+    
+    public static int serializeOffsetArrayIntoStream(DataOutputStream oStream, TrustedByteArrayOutputStream byteStream,
+            int noOfElements, int maxOffset, int[] offsetPos, byte serializationVersion) throws IOException {
         int offsetPosition = (byteStream.size());
         byte[] offsetArr = null;
         boolean useInt = true;
-        if (PArrayDataType.useShortForOffsetArray(maxOffset)) {
+        if (PArrayDataType.useShortForOffsetArray(maxOffset, serializationVersion)) {
             offsetArr = new byte[PArrayDataType.initOffsetArray(noOfElements, Bytes.SIZEOF_SHORT)];
             useInt = false;
         } else {
@@ -1034,7 +899,8 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
             }
         } else {
             for (int pos : offsetPos) {
-                Bytes.putShort(offsetArr, off, (short)(pos - Short.MAX_VALUE));
+                short val = serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION ? (short)pos : (short)(pos - Short.MAX_VALUE);
+				Bytes.putShort(offsetArr, off, val);
                 off += Bytes.SIZEOF_SHORT;
             }
         }
@@ -1043,18 +909,11 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         return noOfElements;
     }
 
-    public static void serializeHeaderInfoIntoBuffer(ByteBuffer buffer, int noOfElements) {
-        // No of elements
-        buffer.putInt(noOfElements);
-        // Version of the array
-        buffer.put(ARRAY_SERIALIZATION_VERSION);
-    }
-
-    public static void serializeHeaderInfoIntoStream(DataOutputStream oStream, int noOfElements) throws IOException {
+    public static void serializeHeaderInfoIntoStream(DataOutputStream oStream, int noOfElements, byte serializationVersion) throws IOException {
         // No of elements
         oStream.writeInt(noOfElements);
         // Version of the array
-        oStream.write(ARRAY_SERIALIZATION_VERSION);
+        oStream.write(serializationVersion);
     }
 
     public static int initOffsetArray(int noOfElements, int baseSize) {
@@ -1228,91 +1087,4 @@ public abstract class PArrayDataType<T> extends PDataType<T> {
         buf.append(']');
         return buf.toString();
     }
-
-    // FIXME: remove this duplicate code
-    static public class PArrayDataTypeBytesArrayBuilder<T> {
-        static private final int BYTE_ARRAY_DEFAULT_SIZE = 128;
-
-        private PDataType baseType;
-        private SortOrder sortOrder;
-        private List<Integer> offsetPos;
-        private TrustedByteArrayOutputStream byteStream;
-        private DataOutputStream oStream;
-        private int nulls;
-
-        public PArrayDataTypeBytesArrayBuilder(PDataType baseType, SortOrder sortOrder) {
-            this.baseType = baseType;
-            this.sortOrder = sortOrder;
-            offsetPos = new LinkedList<Integer>();
-            byteStream = new TrustedByteArrayOutputStream(BYTE_ARRAY_DEFAULT_SIZE);
-            oStream = new DataOutputStream(byteStream);
-            nulls = 0;
-        }
-
-        private void close() {
-            try {
-                if (byteStream != null) byteStream.close();
-                if (oStream != null) oStream.close();
-                byteStream = null;
-                oStream = null;
-            } catch (IOException ioe) {}
-        }
-
-        public boolean appendElem(byte[] bytes) {
-            return appendElem(bytes, 0, bytes.length);
-        }
-
-        public boolean appendElem(byte[] bytes, int offset, int len) {
-            if (oStream == null || byteStream == null) return false;
-            try {
-                if (!baseType.isFixedWidth()) {
-                    if (len == 0) {
-                        offsetPos.add(byteStream.size());
-                        nulls++;
-                    } else {
-                        nulls = serializeNulls(oStream, nulls);
-                        offsetPos.add(byteStream.size());
-                        if (sortOrder == SortOrder.DESC) {
-                            SortOrder.invert(bytes, offset, bytes, offset, len);
-                            offset = 0;
-                        }
-                        oStream.write(bytes, offset, len);
-                        oStream.write(getSeparatorByte(true, sortOrder));
-                    }
-                } else {
-                    if (sortOrder == SortOrder.DESC) {
-                        SortOrder.invert(bytes, offset, bytes, offset, len);
-                        offset = 0;
-                    }
-                    oStream.write(bytes, offset, len);
-                }
-                return true;
-            } catch (IOException e) {}
-            return false;
-        }
-
-        public byte[] getBytesAndClose(SortOrder sortOrder) {
-            try {
-                if (!baseType.isFixedWidth()) {
-                    int noOfElements = offsetPos.size();
-                    int[] offsetPosArray = new int[noOfElements];
-                    int index = 0;
-                    for (Integer i : offsetPos) {
-                        offsetPosArray[index] = i;
-                        ++index;
-                    }
-                    PArrayDataType.writeEndSeperatorForVarLengthArray(oStream, sortOrder);
-                    noOfElements = PArrayDataType.serailizeOffsetArrayIntoStream(oStream, byteStream, noOfElements,
-                            offsetPosArray[offsetPosArray.length - 1], offsetPosArray);
-                    serializeHeaderInfoIntoStream(oStream, noOfElements);
-                }
-                ImmutableBytesWritable ptr = new ImmutableBytesWritable();
-                ptr.set(byteStream.getBuffer(), 0, byteStream.size());
-                return ByteUtil.copyKeyBytesIfNecessary(ptr);
-            } catch (IOException e) {} finally {
-                close();
-            }
-            return null;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
new file mode 100644
index 0000000..7a6ea91
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeDecoder.java
@@ -0,0 +1,102 @@
+/*
+ * 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.types;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnValueDecoder;
+import org.apache.phoenix.schema.tuple.Tuple;
+import org.apache.phoenix.util.ByteUtil;
+
+
+public class PArrayDataTypeDecoder implements ColumnValueDecoder {
+    
+    @Override
+    public boolean decode(ImmutableBytesWritable ptr, int index) {
+        return PArrayDataTypeDecoder.positionAtArrayElement(ptr, index, PVarbinary.INSTANCE, null);
+    }
+
+    public static boolean positionAtArrayElement(Tuple tuple, ImmutableBytesWritable ptr, int index,
+            Expression arrayExpr, PDataType pDataType, Integer maxLen) {
+        if (!arrayExpr.evaluate(tuple, ptr)) {
+            return false;
+        } else if (ptr.getLength() == 0) { return true; }
+    
+        // Given a ptr to the entire array, set ptr to point to a particular element within that array
+        // given the type of an array element (see comments in PDataTypeForArray)
+        return positionAtArrayElement(ptr, index - 1, pDataType, maxLen);
+    }
+
+    public static boolean positionAtArrayElement(ImmutableBytesWritable ptr, int arrayIndex, PDataType baseDataType,
+            Integer byteSize) {
+        byte[] bytes = ptr.get();
+        int initPos = ptr.getOffset();
+        if (!baseDataType.isFixedWidth()) {
+        	byte serializationVersion = bytes[ptr.getOffset() + ptr.getLength() - Bytes.SIZEOF_BYTE];
+            int noOfElements = Bytes.toInt(bytes,
+                    (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + Bytes.SIZEOF_INT)), Bytes.SIZEOF_INT);
+            boolean useShort = true;
+            if (noOfElements < 0) {
+                noOfElements = -noOfElements;
+                useShort = false;
+            }
+            if (arrayIndex >= noOfElements) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                return false;
+            }
+    
+            int indexOffset = Bytes.toInt(bytes,
+                    (ptr.getOffset() + ptr.getLength() - (Bytes.SIZEOF_BYTE + 2 * Bytes.SIZEOF_INT))) + ptr.getOffset();
+            // Skip those many offsets as given in the arrayIndex
+            // If suppose there are 5 elements in the array and the arrayIndex = 3
+            // This means we need to read the 4th element of the array
+            // So inorder to know the length of the 4th element we will read the offset of 4th element and the
+            // offset of 5th element.
+            // Subtracting the offset of 5th element and 4th element will give the length of 4th element
+            // So we could just skip reading the other elements.
+            int currOffset = PArrayDataType.getSerializedOffset(bytes, arrayIndex, useShort, indexOffset, serializationVersion);
+            if (currOffset<0) {
+            	ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+                return false;
+            }
+            int elementLength = 0;
+            if (arrayIndex == (noOfElements - 1)) {
+                int separatorBytes =  serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION ? 3 : 0;
+                elementLength = (bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : indexOffset
+                        - (currOffset + initPos) - separatorBytes;
+            } else {
+                int separatorByte =  serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION ? 1 : 0;
+                elementLength = (bytes[currOffset + initPos] == QueryConstants.SEPARATOR_BYTE || bytes[currOffset + initPos] == QueryConstants.DESC_SEPARATOR_BYTE) ? 0 : PArrayDataType.getOffset(bytes,
+                        arrayIndex + 1, useShort, indexOffset, serializationVersion) - currOffset - separatorByte;
+            }
+            ptr.set(bytes, currOffset + initPos, elementLength);
+        } else {
+            int elemByteSize = (byteSize == null ? baseDataType.getByteSize() : byteSize);
+            int offset = arrayIndex * elemByteSize;
+            if (offset >= ptr.getLength()) {
+                ptr.set(ByteUtil.EMPTY_BYTE_ARRAY);
+            } else {
+                ptr.set(bytes, ptr.getOffset() + offset, elemByteSize);
+            }
+        }
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
new file mode 100644
index 0000000..bb293bb
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/types/PArrayDataTypeEncoder.java
@@ -0,0 +1,170 @@
+/*
+ * 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.types;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.phoenix.schema.ColumnValueEncoder;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.TrustedByteArrayOutputStream;
+
+public class PArrayDataTypeEncoder implements ColumnValueEncoder {
+    static private final int BYTE_ARRAY_DEFAULT_SIZE = 128;
+
+    private PDataType baseType;
+    private SortOrder sortOrder;
+    private List<Integer> offsetPos;
+    private TrustedByteArrayOutputStream byteStream;
+    private DataOutputStream oStream;
+    private int nulls;
+    private byte serializationVersion;
+    private boolean rowKeyOrderOptimizable;
+
+    public PArrayDataTypeEncoder(PDataType baseType, SortOrder sortOrder) {
+        this(new TrustedByteArrayOutputStream(BYTE_ARRAY_DEFAULT_SIZE), new LinkedList<Integer>(), baseType, sortOrder, true);
+    }
+    
+    public PArrayDataTypeEncoder(TrustedByteArrayOutputStream byteStream, DataOutputStream oStream,
+            int numElements, PDataType baseType, SortOrder sortOrder, boolean rowKeyOrderOptimizable, byte serializationVersion) {
+        this(byteStream, oStream, new ArrayList<Integer>(numElements), baseType, sortOrder, rowKeyOrderOptimizable, serializationVersion);
+    }
+    
+    public PArrayDataTypeEncoder(TrustedByteArrayOutputStream byteStream, DataOutputStream oStream,
+            int numElements, PDataType baseType, SortOrder sortOrder, boolean rowKeyOrderOptimizable) {
+        this(byteStream, oStream, new ArrayList<Integer>(numElements), baseType, sortOrder, rowKeyOrderOptimizable, PArrayDataType.SORTABLE_SERIALIZATION_VERSION);
+    }
+    
+    public PArrayDataTypeEncoder(TrustedByteArrayOutputStream byteStream, 
+            List<Integer> offsetPos, PDataType baseType, SortOrder sortOrder, boolean rowKeyOrderOptimizable) {
+        this(byteStream, new DataOutputStream(byteStream), offsetPos, baseType, sortOrder, rowKeyOrderOptimizable, PArrayDataType.SORTABLE_SERIALIZATION_VERSION);
+    }
+    
+    public PArrayDataTypeEncoder(TrustedByteArrayOutputStream byteStream, DataOutputStream oStream,
+            List<Integer> offsetPos, PDataType baseType, SortOrder sortOrder, boolean rowKeyOrderOptimizable, byte serializationVersion) {
+        this.baseType = baseType;
+        this.sortOrder = sortOrder;
+        this.offsetPos = offsetPos;
+        this.byteStream = byteStream;
+        this.oStream = oStream;
+        this.nulls = 0;
+        this.serializationVersion = serializationVersion;
+        this.rowKeyOrderOptimizable = rowKeyOrderOptimizable;
+    }
+
+    private void close() {
+        try {
+            if (byteStream != null) byteStream.close();
+            if (oStream != null) oStream.close();
+            byteStream = null;
+            oStream = null;
+        } catch (IOException ioe) {}
+    }
+    
+    // used to represent the absence of a value 
+    @Override
+    public void appendAbsentValue() {
+        if (serializationVersion == PArrayDataType.IMMUTABLE_SERIALIZATION_VERSION && !baseType.isFixedWidth()) {
+            offsetPos.add(-byteStream.size());
+            nulls++;
+        }
+        else {
+            throw new UnsupportedOperationException("Cannot represent an absent element");
+        }
+    }
+
+    public void appendValue(byte[] bytes) {
+        appendValue(bytes, 0, bytes.length);
+    }
+
+    @Override
+    public void appendValue(byte[] bytes, int offset, int len) {
+        try {
+            // track the offset position here from the size of the byteStream
+            if (!baseType.isFixedWidth()) {
+                // Any variable length array would follow the below order
+                // Every element would be seperated by a seperator byte '0'
+                // Null elements are counted and once a first non null element appears we
+                // write the count of the nulls prefixed with a seperator byte
+                // Trailing nulls are not taken into account
+                // The last non null element is followed by two seperator bytes
+                // For eg
+                // a, b, null, null, c, null would be 
+                // 65 0 66 0 0 2 67 0 0 0
+                // a null null null b c null d would be
+                // 65 0 0 3 66 0 67 0 0 1 68 0 0 0
+                if (len == 0) {
+                    offsetPos.add(byteStream.size());
+                    nulls++;
+                } else {
+                    nulls = PArrayDataType.serializeNulls(oStream, nulls);
+                    offsetPos.add(byteStream.size());
+                    if (sortOrder == SortOrder.DESC) {
+                        SortOrder.invert(bytes, offset, bytes, offset, len);
+                        offset = 0;
+                    }
+                    oStream.write(bytes, offset, len);
+                    if (serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION) {
+                        oStream.write(PArrayDataType.getSeparatorByte(rowKeyOrderOptimizable, sortOrder));
+                    }
+                }
+            } else {
+                // No nulls for fixed length
+                if (sortOrder == SortOrder.DESC) {
+                    SortOrder.invert(bytes, offset, bytes, offset, len);
+                    offset = 0;
+                }
+                oStream.write(bytes, offset, len);
+            }
+        } catch (IOException e) {}
+    }
+
+    @Override
+    public byte[] encode() {
+        try {
+            if (!baseType.isFixedWidth()) {
+                int noOfElements = offsetPos.size();
+                int[] offsetPosArray = new int[noOfElements];
+                int index = 0;
+                for (Integer i : offsetPos) {
+                    offsetPosArray[index] = i;
+                    ++index;
+                }
+                if (serializationVersion == PArrayDataType.SORTABLE_SERIALIZATION_VERSION) {
+                    // Double seperator byte to show end of the non null array
+                    PArrayDataType.writeEndSeperatorForVarLengthArray(oStream, sortOrder, rowKeyOrderOptimizable);
+                }
+                noOfElements = PArrayDataType.serializeOffsetArrayIntoStream(oStream, byteStream, noOfElements,
+                        offsetPosArray[offsetPosArray.length - 1], offsetPosArray, serializationVersion);
+                PArrayDataType.serializeHeaderInfoIntoStream(oStream, noOfElements, serializationVersion);
+            }
+            ImmutableBytesWritable ptr = new ImmutableBytesWritable();
+            ptr.set(byteStream.getBuffer(), 0, byteStream.size());
+            return ByteUtil.copyKeyBytesIfNecessary(ptr);
+        } catch (IOException e) {} finally {
+            close();
+        }
+        return null;
+    }
+    
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..fb6baf0
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/EncodedColumnsUtil.java
@@ -0,0 +1,205 @@
+/*
+ * 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.checkNotNull;
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.coprocessor.BaseScannerRegionObserver;
+import org.apache.phoenix.expression.DelegateExpression;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.expression.LiteralExpression;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.tuple.Tuple;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Maps;
+
+public class EncodedColumnsUtil {
+
+    public static boolean usesEncodedColumnNames(PTable table) {
+        return usesEncodedColumnNames(table.getEncodingScheme());
+    }
+    
+    public static boolean usesEncodedColumnNames(QualifierEncodingScheme encodingScheme) {
+        return encodingScheme != null && encodingScheme != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+    }
+    
+    public static void setColumns(PColumn column, PTable table, Scan scan) {
+    	if (table.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS) {
+            // 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 {
+            if (column.getColumnQualifierBytes() != null) {
+                scan.addColumn(column.getFamilyName().getBytes(), column.getColumnQualifierBytes());
+            }
+        }
+    }
+    
+    public static final boolean useNewValueColumnQualifier(Scan s) {
+        // null check for backward compatibility
+        return s.getAttribute(BaseScannerRegionObserver.USE_NEW_VALUE_COLUMN_QUALIFIER) == null ? false : true;
+    }
+    
+    public static QualifierEncodingScheme getQualifierEncodingScheme(Scan s) {
+        // null check for backward compatibility
+        return s.getAttribute(BaseScannerRegionObserver.QUALIFIER_ENCODING_SCHEME) == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : QualifierEncodingScheme.fromSerializedValue(s.getAttribute(BaseScannerRegionObserver.QUALIFIER_ENCODING_SCHEME)[0]);
+    }
+    
+    public static ImmutableStorageScheme getImmutableStorageScheme(Scan s) {
+        // null check for backward compatibility
+        return s.getAttribute(BaseScannerRegionObserver.IMMUTABLE_STORAGE_ENCODING_SCHEME) == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ImmutableStorageScheme.fromSerializedValue(s.getAttribute(BaseScannerRegionObserver.IMMUTABLE_STORAGE_ENCODING_SCHEME)[0]);
+    }
+
+    /**
+     * @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);
+    }
+    
+    /**
+     * @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(QualifierEncodingScheme encodingScheme) {
+        return usesEncodedColumnNames(encodingScheme) ? 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 Pair<Integer, Integer> getMinMaxQualifiersFromScan(Scan scan) {
+        Integer minQ = null, maxQ = null;
+        byte[] minQualifier = scan.getAttribute(BaseScannerRegionObserver.MIN_QUALIFIER);
+        if (minQualifier != null) {
+            minQ = Bytes.toInt(minQualifier);
+        }
+        byte[] maxQualifier = scan.getAttribute(BaseScannerRegionObserver.MAX_QUALIFIER);
+        if (maxQualifier != null) {
+            maxQ = Bytes.toInt(maxQualifier);
+        }
+        if (minQualifier == null) {
+            return null;
+        }
+        return new Pair<>(minQ, maxQ);
+    }
+
+    public static boolean setQualifierRanges(PTable table) {
+        return table.getImmutableStorageScheme() != null
+                && table.getImmutableStorageScheme() == ImmutableStorageScheme.ONE_CELL_PER_COLUMN
+                && usesEncodedColumnNames(table) && !table.isTransactional()
+                && !ScanUtil.hasDynamicColumns(table);
+    }
+
+    public static boolean useQualifierAsIndex(Pair<Integer, Integer> minMaxQualifiers) {
+        return minMaxQualifiers != null;
+    }
+
+    public static Map<String, Pair<Integer, Integer>> getFamilyQualifierRanges(PTable table) {
+        checkNotNull(table);
+        QualifierEncodingScheme encodingScheme = table.getEncodingScheme();
+        Preconditions.checkArgument(encodingScheme != NON_ENCODED_QUALIFIERS);
+        if (table.getEncodedCQCounter() != null) {
+            Map<String, Integer> values = table.getEncodedCQCounter().values();
+            Map<String, Pair<Integer, Integer>> toReturn = Maps.newHashMapWithExpectedSize(values.size());
+            for (Entry<String, Integer> e : values.entrySet()) {
+                Integer lowerBound = QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+                Integer upperBound = e.getValue() - 1;
+                if (lowerBound > upperBound) {
+                    lowerBound = upperBound;
+                }
+                toReturn.put(e.getKey(), new Pair<>(lowerBound, upperBound));
+            }
+            return toReturn;
+        }
+        return Collections.emptyMap();
+    }
+    
+    public static byte[] getColumnQualifierBytes(String columnName, Integer numberBasedQualifier, PTable table, boolean isPk) {
+        QualifierEncodingScheme encodingScheme = table.getEncodingScheme();
+        return getColumnQualifierBytes(columnName, numberBasedQualifier, encodingScheme, isPk);
+    }
+    
+    public static byte[] getColumnQualifierBytes(String columnName, Integer numberBasedQualifier, QualifierEncodingScheme encodingScheme, boolean isPk) {
+        if (isPk) {
+            return null;
+        }
+        if (encodingScheme == null || encodingScheme == NON_ENCODED_QUALIFIERS) {
+            return Bytes.toBytes(columnName);
+        }
+        return encodingScheme.encode(numberBasedQualifier);
+    }
+    
+    public static Expression[] createColumnExpressionArray(int maxEncodedColumnQualifier) {
+        // reserve the first position and offset maxEncodedColumnQualifier by ENCODED_CQ_COUNTER_INITIAL_VALUE (which is the minimum encoded column qualifier)
+        int numElements = maxEncodedColumnQualifier - QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE + 2;
+        Expression[] colValues = new Expression[numElements];
+        Arrays.fill(colValues, new DelegateExpression(LiteralExpression.newConstant(null)) {
+                   @Override
+                   public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+                       return false;
+                   }
+               });
+        // 0 is a reserved position, set it to a non-null value so that we can represent absence of a value using a negative offset
+        colValues[0]=LiteralExpression.newConstant(QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+        return colValues;
+    }
+
+    public static boolean isReservedColumnQualifier(int number) {
+        if (number < 0) {
+            throw new IllegalArgumentException("Negative column qualifier" + number + " not allowed ");
+        }
+        return number < QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+    }
+    
+    public static boolean isPossibleToUseEncodedCQFilter(QualifierEncodingScheme encodingScheme,
+            ImmutableStorageScheme storageScheme) {
+        return EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme)
+                && storageScheme == ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
+    }
+
+}


[14/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
index d3bbe23..1084f14 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryIT.java
@@ -62,15 +62,15 @@ import org.junit.Test;
  */
 public class QueryIT extends BaseQueryIT {
     
-    public QueryIT(String indexDDL) {
-        super(indexDDL);
+    public QueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Test
     public void testIntFilter() throws Exception {
         String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_INTEGER) " +
@@ -89,10 +89,10 @@ public class QueryIT extends BaseQueryIT {
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 6);
         props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         upsertConn = DriverManager.getConnection(url, props);
-        analyzeTable(upsertConn, "ATABLE");
+        analyzeTable(upsertConn, tableName);
         upsertConn.close();
 
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer >= ?";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer >= ?";
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
         PreparedStatement statement = conn.prepareStatement(query);
@@ -101,19 +101,19 @@ public class QueryIT extends BaseQueryIT {
         ResultSet rs = statement.executeQuery();
         assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW7, ROW8, ROW9));
 
-        query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer < 2";
+        query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer < 2";
         statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         rs = statement.executeQuery();
         assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW1, ROW4));
 
-        query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer <= 2";
+        query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer <= 2";
         statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         rs = statement.executeQuery();
         assertValueEqualsResultSet(rs, Arrays.<Object>asList(ROW1, ROW2, ROW4));
 
-        query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_integer >=9";
+        query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_integer >=9";
         statement = conn.prepareStatement(query);
         statement.setString(1, tenantId);
         rs = statement.executeQuery();
@@ -124,13 +124,8 @@ public class QueryIT extends BaseQueryIT {
     }
     
     @Test
-    public void testEmptyStringValue() throws Exception {
-        testNoStringValue("");
-    }
-
-    @Test
     public void testToDateOnString() throws Exception { // TODO: test more conversion combinations
-        String query = "SELECT a_string FROM aTable WHERE organization_id=? and a_integer = 5";
+        String query = "SELECT a_string FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -151,7 +146,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testColumnOnBothSides() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and a_string = b_string";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and a_string = b_string";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -167,50 +162,9 @@ public class QueryIT extends BaseQueryIT {
         }
     }
 
-    private void testNoStringValue(String value) throws Exception {
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        PreparedStatement stmt = upsertConn.prepareStatement(
-                "upsert into ATABLE VALUES (?, ?, ?)"); // without specifying columns
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW5);
-        stmt.setString(3, value);
-        stmt.execute(); // should commit too
-        upsertConn.close();
-        
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        Connection conn1 = DriverManager.getConnection(getUrl(), props);
-        analyzeTable(conn1, "ATABLE");
-        conn1.close();
-        
-        String query = "SELECT a_string, b_string FROM aTable WHERE organization_id=? and a_integer = 5";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            PreparedStatement statement = conn.prepareStatement(query);
-            statement.setString(1, tenantId);
-            ResultSet rs = statement.executeQuery();
-            assertTrue (rs.next());
-            assertEquals(null, rs.getString(1));
-            assertTrue(rs.wasNull());
-            assertEquals(C_VALUE, rs.getString("B_string"));
-            assertFalse(rs.next());
-        } finally {
-            conn.close();
-        }
-    }
-
-    @Test
-    public void testNullStringValue() throws Exception {
-        testNoStringValue(null);
-    }
-    
     @Test
     public void testDateInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE a_date IN (?,?) AND a_integer < 4";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE a_date IN (?,?) AND a_integer < 4";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -230,8 +184,8 @@ public class QueryIT extends BaseQueryIT {
     @Test
     public void testTimestamp() throws Exception {
         String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_TIMESTAMP) " +
@@ -251,12 +205,12 @@ public class QueryIT extends BaseQueryIT {
         
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);       
         Connection conn1 = DriverManager.getConnection(url, props);
-        analyzeTable(conn1, "ATABLE");
+        analyzeTable(conn1, tableName);
         conn1.close();
         
         updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_TIMESTAMP," +
@@ -276,7 +230,7 @@ public class QueryIT extends BaseQueryIT {
         assertTrue(compare(CompareOp.GREATER, new ImmutableBytesWritable(ts2), new ImmutableBytesWritable(ts1)));
         assertFalse(compare(CompareOp.GREATER, new ImmutableBytesWritable(ts1), new ImmutableBytesWritable(ts1)));
 
-        String query = "SELECT entity_id, a_timestamp, a_time FROM aTable WHERE organization_id=? and a_timestamp > ?";
+        String query = "SELECT entity_id, a_timestamp, a_time FROM " + tableName + " WHERE organization_id=? and a_timestamp > ?";
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
         try {
@@ -296,7 +250,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testSimpleInListStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND a_integer IN (2,4)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND a_integer IN (2,4)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -312,7 +266,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testPartiallyQualifiedRVCInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE (a_integer,a_string) IN ((2,'a'),(5,'b'))";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE (a_integer,a_string) IN ((2,'a'),(5,'b'))";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -327,7 +281,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testFullyQualifiedRVCInList() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE (a_integer,a_string, organization_id,entity_id) IN ((2,'a',:1,:2),(5,'b',:1,:3))";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE (a_integer,a_string, organization_id,entity_id) IN ((2,'a',:1,:2),(5,'b',:1,:3))";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -345,7 +299,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testOneInListStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND b_string IN (?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND b_string IN (?)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -369,7 +323,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testMixedTypeInListStatement() throws Exception {
-        String query = "SELECT entity_id FROM ATABLE WHERE organization_id=? AND x_long IN (5, ?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? AND x_long IN (5, ?)";
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(url, props);
@@ -391,7 +345,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testIsNull() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE X_DECIMAL is null";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE X_DECIMAL is null";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -418,7 +372,7 @@ public class QueryIT extends BaseQueryIT {
 
     @Test
     public void testIsNotNull() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE X_DECIMAL is not null";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE X_DECIMAL is not null";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -442,11 +396,11 @@ public class QueryIT extends BaseQueryIT {
         int counter=0;
         String[] answers = new String[]{"00D300000000XHP5bar","a5bar","15bar","5bar","5bar"};
         String[] queries = new String[] { 
-        		"SELECT  organization_id || 5 || 'bar' FROM atable limit 1",
-        		"SELECT a_string || 5 || 'bar' FROM atable  order by a_string  limit 1",
-        		"SELECT a_integer||5||'bar' FROM atable order by a_integer  limit 1",
-        		"SELECT x_decimal||5||'bar' FROM atable limit 1",
-        		"SELECT x_long||5||'bar' FROM atable limit 1"
+        		"SELECT  organization_id || 5 || 'bar' FROM " + tableName + " limit 1",
+        		"SELECT a_string || 5 || 'bar' FROM " + tableName + "  order by a_string  limit 1",
+        		"SELECT a_integer||5||'bar' FROM " + tableName + " order by a_integer  limit 1",
+        		"SELECT x_decimal||5||'bar' FROM " + tableName + " limit 1",
+        		"SELECT x_long||5||'bar' FROM " + tableName + " limit 1"
         };
 
         for (String query : queries) {
@@ -468,7 +422,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testRowKeySingleIn() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id IN (?,?,?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and entity_id IN (?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -494,7 +448,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testRowKeyMultiIn() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id IN (?,?,?) and a_string IN (?,?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and entity_id IN (?,?,?) and a_string IN (?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -519,7 +473,7 @@ public class QueryIT extends BaseQueryIT {
     
     @Test
     public void testColumnAliasMapping() throws Exception {
-        String query = "SELECT a.a_string, aTable.b_string FROM aTable a WHERE ?=organization_id and 5=a_integer ORDER BY a_string, b_string";
+        String query = "SELECT a.a_string, " + tableName + ".b_string FROM " + tableName + " a WHERE ?=organization_id and 5=a_integer ORDER BY a_string, b_string";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
index 1573533..8b320fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ReadIsolationLevelIT.java
@@ -42,7 +42,7 @@ public class ReadIsolationLevelIT extends BaseClientManagedTimeIT {
 
     protected static void initTableValues(long ts, byte[][] splits) throws Exception {
         String tenantId = getOrganizationId();
-        ensureTableCreated(getUrl(),ATABLE_NAME, ATABLE_NAME, splits, ts-2);
+        ensureTableCreated(getUrl(),ATABLE_NAME, ATABLE_NAME, splits, ts-2, null);
 
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..b7d67f2 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
@@ -66,7 +66,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorInWhereWithEqualsExpression() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) = (7, 5)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -91,7 +91,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorInWhereWithGreaterThanExpression() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) >= (4, 4)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -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();
         }
@@ -117,7 +117,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorInWhereWithUnEqualNumberArgs() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer, y_integer) >= (7, 5)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -143,7 +143,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testBindVarsInRowValueConstructor() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) = (?, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -170,7 +170,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnLHSAndLiteralExpressionOnRHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) >= 7";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -194,7 +194,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnRHSLiteralExpressionOnLHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND 7 <= (a_integer, x_integer)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -218,7 +218,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnLHSBuiltInFunctionOperatingOnIntegerLiteralRHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) >= to_number('7')";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -242,7 +242,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnRHSWithBuiltInFunctionOperatingOnIntegerLiteralOnLHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND to_number('7') <= (a_integer, x_integer)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -266,7 +266,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnLHSWithBuiltInFunctionOperatingOnColumnRefOnRHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts - 1, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts - 1, getUrl(), null);
         String upsertQuery = "UPSERT INTO aTable(organization_id, entity_id, a_string) values (?, ?, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
@@ -332,7 +332,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRowValueConstructorOnRHSWithBuiltInFunctionOperatingOnColumnRefOnLHS() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts - 1, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts - 1, getUrl(), null);
         String upsertQuery = "UPSERT INTO aTable(organization_id, entity_id, a_string) values (?, ?, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
@@ -654,7 +654,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithNonLeadingPkColsOfTypesIntegerAndString() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, a_string FROM aTable WHERE ?=organization_id  AND (a_integer, a_string) <= (5, 'a')";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -680,7 +680,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithNonLeadingPkColsOfTypesTimeStampAndString() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String updateStmt = 
             "upsert into " +
             "ATABLE(" +
@@ -724,7 +724,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testNestedRVCBasic() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         //all the three queries should return the same rows.
         String[] queries = {"SELECT organization_id, entity_id, a_string FROM aTable WHERE ((organization_id, entity_id), a_string) >= ((?, ?), ?)",
                             "SELECT organization_id, entity_id, a_string FROM aTable WHERE (organization_id, entity_id, a_string) >= (?, ?, ?)",
@@ -762,7 +762,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithInListClausePossibleNullValues() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         //we have a row present in aTable where x_integer = 5 and y_integer = NULL which gets translated to 0 when retriving from HBase. 
         String query = "SELECT x_integer, y_integer FROM aTable WHERE ? = organization_id AND (x_integer, y_integer) IN ((5))";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -784,7 +784,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithInListClauseUsingSubsetOfPKColsInOrder() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         // Though we have a row present in aTable where organization_id = tenantId and  x_integer = 5,
         // we'd also need to have an entity_id that is null (which we don't have).
         String query = "SELECT organization_id, entity_id FROM aTable WHERE (organization_id, entity_id) IN (('" + tenantId + "')) AND x_integer = 5";
@@ -814,7 +814,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
     public void testRVCWithCeilAndFloorNeededForDecimal() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT a_integer, x_integer FROM aTable WHERE ?=organization_id  AND (a_integer, x_integer) < (8.6, 4.5) AND (a_integer, x_integer) > (6.8, 4)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -841,7 +841,7 @@ public class RowValueConstructorIT extends BaseClientManagedTimeIT {
         String tenantId = getOrganizationId();
         Date dateUpserted = DateUtil.parseDate("2012-01-01 14:25:28");
         dateUpserted = new Date(dateUpserted.getTime() + 660); // this makes the dateUpserted equivalent to 2012-01-01 14:25:28.660
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), dateUpserted, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), dateUpserted, ts, getUrl(), null);
         String query = "SELECT a_integer, a_date FROM aTable WHERE ?=organization_id  AND (a_integer, a_date) <= (9, ?) AND (a_integer, a_date) >= (6, ?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
index 9b28bad..b0ce8cd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ScanQueryIT.java
@@ -20,7 +20,6 @@ package org.apache.phoenix.end2end;
 import static org.apache.phoenix.util.TestUtil.A_VALUE;
 import static org.apache.phoenix.util.TestUtil.B_VALUE;
 import static org.apache.phoenix.util.TestUtil.C_VALUE;
-import static org.apache.phoenix.util.TestUtil.E_VALUE;
 import static org.apache.phoenix.util.TestUtil.ROW1;
 import static org.apache.phoenix.util.TestUtil.ROW2;
 import static org.apache.phoenix.util.TestUtil.ROW3;
@@ -39,10 +38,8 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Properties;
 import java.util.Set;
 
@@ -53,7 +50,6 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Lists;
 import com.google.common.primitives.Doubles;
 import com.google.common.primitives.Floats;
 
@@ -66,13 +62,13 @@ public class ScanQueryIT extends BaseQueryIT {
         return QueryIT.data();
     }
 
-    public ScanQueryIT(String indexDDL) {
-        super(indexDDL);
+    public ScanQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Test
     public void testScan() throws Exception {
-        String query = "SELECT a_string, /* comment ok? */ b_string FROM aTable WHERE ?=organization_id and 5=a_integer";
+        String query = "SELECT a_string, /* comment ok? */ b_string FROM " + tableName + " WHERE ?=organization_id and 5=a_integer";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -91,7 +87,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testScanByByteValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_byte FROM aTable WHERE ?=organization_id and 1=a_byte";
+        String query = "SELECT a_string, b_string, a_byte FROM " + tableName + " WHERE ?=organization_id and 1=a_byte";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -111,7 +107,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testScanByShortValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_short FROM aTable WHERE ?=organization_id and 128=a_short";
+        String query = "SELECT a_string, b_string, a_short FROM " + tableName + " WHERE ?=organization_id and 128=a_short";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -131,7 +127,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testScanByFloatValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_float FROM aTable WHERE ?=organization_id and ?=a_float";
+        String query = "SELECT a_string, b_string, a_float FROM " + tableName + " WHERE ?=organization_id and ?=a_float";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -152,7 +148,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testScanByUnsignedFloatValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_unsigned_float FROM aTable WHERE ?=organization_id and ?=a_unsigned_float";
+        String query = "SELECT a_string, b_string, a_unsigned_float FROM " + tableName + " WHERE ?=organization_id and ?=a_unsigned_float";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -173,7 +169,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testScanByDoubleValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_double FROM aTable WHERE ?=organization_id and ?=a_double";
+        String query = "SELECT a_string, b_string, a_double FROM " + tableName + " WHERE ?=organization_id and ?=a_double";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -194,7 +190,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testScanByUnsigned_DoubleValue() throws Exception {
-        String query = "SELECT a_string, b_string, a_unsigned_double FROM aTable WHERE ?=organization_id and ?=a_unsigned_double";
+        String query = "SELECT a_string, b_string, a_unsigned_double FROM " + tableName + " WHERE ?=organization_id and ?=a_unsigned_double";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -215,7 +211,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testAllScan() throws Exception {
-        String query = "SELECT ALL a_string, b_string FROM aTable WHERE ?=organization_id and 5=a_integer";
+        String query = "SELECT ALL a_string, b_string FROM " + tableName + " WHERE ?=organization_id and 5=a_integer";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -234,7 +230,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testDistinctScan() throws Exception {
-        String query = "SELECT DISTINCT a_string FROM aTable WHERE organization_id=?";
+        String query = "SELECT DISTINCT a_string FROM " + tableName + " WHERE organization_id=?";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -256,7 +252,7 @@ public class ScanQueryIT extends BaseQueryIT {
 
     @Test
     public void testDistinctLimitScan() throws Exception {
-        String query = "SELECT DISTINCT a_string FROM aTable WHERE organization_id=? LIMIT 1";
+        String query = "SELECT DISTINCT a_string FROM " + tableName + " WHERE organization_id=? LIMIT 1";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -274,7 +270,7 @@ public class ScanQueryIT extends BaseQueryIT {
 
     @Test
     public void testInListSkipScan() throws Exception {
-        String query = "SELECT entity_id, b_string FROM aTable WHERE organization_id=? and entity_id IN (?,?)";
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE organization_id=? and entity_id IN (?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -301,7 +297,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testUnboundRangeScan1() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id <= ?";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id <= ?";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -335,7 +331,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testUnboundRangeScan2() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id >= ?";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id >= ?";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -369,7 +365,7 @@ public class ScanQueryIT extends BaseQueryIT {
     
     @Test
     public void testUpperLowerBoundRangeScan() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) > '00A' and substr(entity_id,1,3) < '00C'";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and substr(entity_id,1,3) > '00A' and substr(entity_id,1,3) < '00C'";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -393,7 +389,7 @@ public class ScanQueryIT extends BaseQueryIT {
 
     @Test
     public void testUpperBoundRangeScan() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) >= '00B' ";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and substr(entity_id,1,3) >= '00B' ";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -419,7 +415,7 @@ public class ScanQueryIT extends BaseQueryIT {
 
     @Test
     public void testLowerBoundRangeScan() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and substr(entity_id,1,3) < '00B' ";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and substr(entity_id,1,3) < '00B' ";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -440,57 +436,4 @@ public class ScanQueryIT extends BaseQueryIT {
             conn.close();
         }
     }
-    
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testPointInTimeLimitedScan() throws Exception {
-        // Override value that was set at creation time
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        String upsertStmt =
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_INTEGER) " +
-            "VALUES (?, ?, ?)";
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW1);
-        stmt.setInt(3, 6);
-        stmt.execute(); // should commit too
-        upsertConn.close();
-
-        // Override value again, but should be ignored since it's past the SCN
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3);
-        upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        stmt = upsertConn.prepareStatement(upsertStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW1);
-        stmt.setInt(3, 0);
-        stmt.execute(); // should commit too
-        upsertConn.close();
-        
-        String query = "SELECT a_integer,b_string FROM atable WHERE organization_id=? and a_integer <= 5 limit 2";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement statement = conn.prepareStatement(query);
-        statement.setString(1, tenantId);
-        ResultSet rs = statement.executeQuery();
-        List<List<Object>> expectedResultsA = Lists.newArrayList(
-                Arrays.<Object>asList(2, C_VALUE),
-                Arrays.<Object>asList( 3, E_VALUE));
-        List<List<Object>> expectedResultsB = Lists.newArrayList(
-                Arrays.<Object>asList( 5, C_VALUE),
-                Arrays.<Object>asList(4, B_VALUE));
-        // Since we're not ordering and we may be using a descending index, we don't
-        // know which rows we'll get back.
-        assertOneOfValuesEqualsResultSet(rs, expectedResultsA,expectedResultsB);
-       conn.close();
-    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 85fc9d7..040c92c 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
@@ -40,6 +40,10 @@ import java.util.Random;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HRegionLocation;
+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.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -48,6 +52,7 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
@@ -58,6 +63,7 @@ import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.junit.Before;
 import org.junit.BeforeClass;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -68,20 +74,54 @@ import com.google.common.collect.Maps;
 @RunWith(Parameterized.class)
 public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
     private final String tableDDLOptions;
+    private final boolean columnEncoded;
     private String tableName;
     private String schemaName;
     private String fullTableName;
     private String physicalTableName;
     private final boolean userTableNamespaceMapped;
+    private final boolean mutable;
     
-    public StatsCollectorIT(boolean transactional, boolean userTableNamespaceMapped) {
-        this.tableDDLOptions= transactional ? " TRANSACTIONAL=true" : "";
+    public StatsCollectorIT(boolean mutable, boolean transactional, boolean userTableNamespaceMapped, boolean columnEncoded) {
+        StringBuilder sb = new StringBuilder();
+        if (transactional) {
+            sb.append("TRANSACTIONAL=true");
+        }
+        if (!columnEncoded) {
+            if (sb.length()>0) {
+                sb.append(",");
+            }
+            sb.append("COLUMN_ENCODED_BYTES=0");
+        } else {
+            if (sb.length()>0) {
+                sb.append(",");
+            }
+            sb.append("COLUMN_ENCODED_BYTES=4");
+        }
+        if (!mutable) {
+            if (sb.length()>0) {
+                sb.append(",");
+            }
+            sb.append("IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                sb.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
+        this.tableDDLOptions = sb.toString();
         this.userTableNamespaceMapped = userTableNamespaceMapped;
+        this.columnEncoded = columnEncoded;
+        this.mutable = mutable;
     }
     
-    @Parameters(name="transactional = {0}, isUserTableNamespaceMapped = {1}")
+    @Parameters(name="columnEncoded = {0}, mutable = {1}, transactional = {2}, isUserTableNamespaceMapped = {3}")
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {{false,true}, {false, false}, {true, false}, {true, true}});
+        return Arrays.asList(new Boolean[][] {     
+                { false, false, false, false }, { false, false, false, true }, { false, false, true, false }, { false, false, true, true },
+                // no need to test non column encoded mutable case and this is the same as non column encoded immutable 
+                //{ false, true, false, false }, { false, true, false, true }, { false, true, true, false }, { false, true, true, true }, 
+                { true, false, false, false }, { true, false, false, true }, { true, false, true, false }, { true, false, true, true }, 
+                { true, true, false, false }, { true, true, false, true }, { true, true, true, false }, { true, true, true, true } 
+          });
     }
     
     @BeforeClass
@@ -147,25 +187,28 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
                 "CREATE TABLE " + fullTableName +" ( k VARCHAR PRIMARY KEY, a.v1 VARCHAR, b.v2 VARCHAR ) " + tableDDLOptions + (tableDDLOptions.isEmpty() ? "" : ",") + "SALT_BUCKETS = 3");
         conn.createStatement().execute("UPSERT INTO " + fullTableName + "(k,v1) VALUES('a','123456789')");
         conn.createStatement().execute("UPDATE STATISTICS " + fullTableName);
+                
         ResultSet rs;
         String explainPlan;
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT v2 FROM " + fullTableName + " WHERE v2='foo'");
         explainPlan = QueryUtil.getExplainPlan(rs);
+        // if we are using the ONE_CELL_PER_COLUMN_FAMILY storage scheme, we will have the single kv even though there are no values for col family v2 
+        String stats = columnEncoded && !mutable  ? "4-CHUNK 1 ROWS 38 BYTES" : "3-CHUNK 0 ROWS 0 BYTES";
         assertEquals(
-                "CLIENT 3-CHUNK 0 ROWS 0 BYTES PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
+                "CLIENT " + stats + " PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\n" +
                 "    SERVER FILTER BY B.V2 = 'foo'\n" + 
                 "CLIENT MERGE SORT",
                 explainPlan);
         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 " + physicalTableName + "\n" +
+                "CLIENT 4-CHUNK 1 ROWS " + (columnEncoded ? "28" : "34") + " BYTES PARALLEL 3-WAY FULL SCAN OVER " + physicalTableName + "\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 " + physicalTableName + "\n" +
+                "CLIENT 1-CHUNK 1 ROWS " + (columnEncoded ? "204" : "202") + " BYTES PARALLEL 1-WAY POINT LOOKUP ON 1 KEY OVER " + physicalTableName + "\n" +
                 "CLIENT MERGE SORT",
                 explainPlan);
         
@@ -368,11 +411,13 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
     }
     
     @Test
+    @Ignore //TODO remove this once  https://issues.apache.org/jira/browse/TEPHRA-208 is fixed
     public void testCompactUpdatesStats() throws Exception {
         testCompactUpdatesStats(0, fullTableName);
     }
     
     @Test
+    @Ignore //TODO remove this once  https://issues.apache.org/jira/browse/TEPHRA-208 is fixed
     public void testCompactUpdatesStatsWithMinStatsUpdateFreq() throws Exception {
         testCompactUpdatesStats(QueryServicesOptions.DEFAULT_STATS_UPDATE_FREQ_MS, fullTableName);
     }
@@ -390,6 +435,7 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         Connection conn = getConnection(statsUpdateFreq);
         PreparedStatement stmt;
         conn.createStatement().execute("CREATE TABLE " + tableName + "(k CHAR(1) PRIMARY KEY, v INTEGER, w INTEGER) "
+                + (!tableDDLOptions.isEmpty() ? tableDDLOptions + "," : "") 
                 + HColumnDescriptor.KEEP_DELETED_CELLS + "=" + Boolean.FALSE);
         stmt = conn.prepareStatement("UPSERT INTO " + tableName + " VALUES(?,?,?)");
         for (int i = 0; i < nRows; i++) {
@@ -399,11 +445,13 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
             stmt.executeUpdate();
         }
         conn.commit();
+        
         compactTable(conn, physicalTableName);
-        if (statsUpdateFreq == null) {
+        
+        if (statsUpdateFreq != 0) {
             invalidateStats(conn, tableName);
         } else {
-            // Confirm that when we have a non zero MIN_STATS_UPDATE_FREQ_MS_ATTRIB, after we run
+            // Confirm that when we have a non zero STATS_UPDATE_FREQ_MS_ATTRIB, after we run
             // UPDATATE STATISTICS, the new statistics are faulted in as expected.
             List<KeyRange>keyRanges = getAllSplits(conn, tableName);
             assertNotEquals(nRows+1, keyRanges.size());
@@ -419,20 +467,40 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         conn.commit();
         assertEquals(5, nDeletedRows);
         
+        Scan scan = new Scan();
+        scan.setRaw(true);
+        PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+        try (HTableInterface htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
+            ResultScanner scanner = htable.getScanner(scan);
+            Result result;
+            while ((result = scanner.next())!=null) {
+                System.out.println(result);
+            }
+        }
+        
         compactTable(conn, physicalTableName);
-        if (statsUpdateFreq == null) {
-            invalidateStats(conn, tableName);
+        
+        scan = new Scan();
+        scan.setRaw(true);
+        phxConn = conn.unwrap(PhoenixConnection.class);
+        try (HTableInterface htable = phxConn.getQueryServices().getTable(Bytes.toBytes(tableName))) {
+            ResultScanner scanner = htable.getScanner(scan);
+            Result result;
+            while ((result = scanner.next())!=null) {
+                System.out.println(result);
+            }
         }
         
-        keyRanges = getAllSplits(conn, tableName);
-        if (statsUpdateFreq != null) {
+        if (statsUpdateFreq != 0) {
+            invalidateStats(conn, tableName);
+        } else {
             assertEquals(nRows+1, keyRanges.size());
-            // If we've set MIN_STATS_UPDATE_FREQ_MS_ATTRIB, an UPDATE STATISTICS will invalidate the cache
+            // If we've set STATS_UPDATE_FREQ_MS_ATTRIB, an UPDATE STATISTICS will invalidate the cache
             // and force us to pull over the new stats
             int rowCount = conn.createStatement().executeUpdate("UPDATE STATISTICS " + tableName);
             assertEquals(5, rowCount);
-            keyRanges = getAllSplits(conn, tableName);
         }
+        keyRanges = getAllSplits(conn, tableName);
         assertEquals(nRows/2+1, keyRanges.size());
         ResultSet rs = conn.createStatement().executeQuery("SELECT SUM(GUIDE_POSTS_ROW_COUNT) FROM "
                 + PhoenixDatabaseMetaData.SYSTEM_STATS_NAME + " WHERE PHYSICAL_NAME='" + physicalTableName + "'");
@@ -447,7 +515,8 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         PreparedStatement stmt;
         conn.createStatement().execute(
                 "CREATE TABLE " + fullTableName
-                        + "(k VARCHAR PRIMARY KEY, a.v INTEGER, b.v INTEGER, c.v INTEGER NULL, d.v INTEGER NULL) ");
+                        + "(k VARCHAR PRIMARY KEY, a.v INTEGER, b.v INTEGER, c.v INTEGER NULL, d.v INTEGER NULL) "
+                        + tableDDLOptions );
         stmt = conn.prepareStatement("UPSERT INTO " + fullTableName + " VALUES(?,?, ?, ?, ?)");
         byte[] val = new byte[250];
         for (int i = 0; i < nRows; i++) {
@@ -473,7 +542,7 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         List<KeyRange> keyRanges = getAllSplits(conn, fullTableName);
         assertEquals(26, keyRanges.size());
         rs = conn.createStatement().executeQuery("EXPLAIN SELECT * FROM " + fullTableName);
-        assertEquals("CLIENT 26-CHUNK 25 ROWS 12420 BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
+        assertEquals("CLIENT 26-CHUNK 25 ROWS " + (columnEncoded ? ( mutable ? "12530" : "13902" ) : "12420") + " BYTES PARALLEL 1-WAY FULL SCAN OVER " + physicalTableName,
                 QueryUtil.getExplainPlan(rs));
 
         ConnectionQueryServices services = conn.unwrap(PhoenixConnection.class).getQueryServices();
@@ -485,7 +554,8 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
                 + QueryServices.STATS_GUIDEPOST_WIDTH_BYTES_ATTRIB + "\"=" + Long.toString(1000);
         conn.createStatement().execute(query);
         keyRanges = getAllSplits(conn, fullTableName);
-        assertEquals(12, keyRanges.size());
+        boolean oneCellPerColFamliyStorageScheme = !mutable && columnEncoded;
+        assertEquals(oneCellPerColFamliyStorageScheme ? 13 : 12, keyRanges.size());
 
         rs = conn
                 .createStatement()
@@ -496,25 +566,25 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         assertTrue(rs.next());
         assertEquals("A", rs.getString(1));
         assertEquals(24, rs.getInt(2));
-        assertEquals(12144, rs.getInt(3));
-        assertEquals(11, rs.getInt(4));
+        assertEquals(columnEncoded ? ( mutable ? 12252 : 13624 ) : 12144, rs.getInt(3));
+        assertEquals(oneCellPerColFamliyStorageScheme ? 12 : 11, rs.getInt(4));
 
         assertTrue(rs.next());
         assertEquals("B", rs.getString(1));
-        assertEquals(20, rs.getInt(2));
-        assertEquals(5540, rs.getInt(3));
-        assertEquals(5, rs.getInt(4));
+        assertEquals(oneCellPerColFamliyStorageScheme ? 24 : 20, rs.getInt(2));
+        assertEquals(columnEncoded ? ( mutable ? 5600 : 6972 ) : 5540, rs.getInt(3));
+        assertEquals(oneCellPerColFamliyStorageScheme ? 6 : 5, rs.getInt(4));
 
         assertTrue(rs.next());
         assertEquals("C", rs.getString(1));
         assertEquals(24, rs.getInt(2));
-        assertEquals(6652, rs.getInt(3));
+        assertEquals(columnEncoded ? ( mutable ? 6724 : 6988 ) : 6652, rs.getInt(3));
         assertEquals(6, rs.getInt(4));
 
         assertTrue(rs.next());
         assertEquals("D", rs.getString(1));
         assertEquals(24, rs.getInt(2));
-        assertEquals(6652, rs.getInt(3));
+        assertEquals(columnEncoded ? ( mutable ? 6724 : 6988 ) : 6652, rs.getInt(3));
         assertEquals(6, rs.getInt(4));
 
         assertFalse(rs.next());
@@ -539,7 +609,7 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
         Connection conn = getConnection();
         String ddl = "CREATE TABLE " + fullTableName + " (t_id VARCHAR NOT NULL,\n" + "k1 INTEGER NOT NULL,\n"
                 + "k2 INTEGER NOT NULL,\n" + "C3.k3 INTEGER,\n" + "C2.v1 VARCHAR,\n"
-                + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) split on ('e','j','o')";
+                + "CONSTRAINT pk PRIMARY KEY (t_id, k1, k2)) " + tableDDLOptions + " split on ('e','j','o')";
         conn.createStatement().execute(ddl);
         String[] strings = { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l", "m", "n", "o", "p", "q", "r",
                 "s", "t", "u", "v", "w", "x", "y", "z" };
@@ -559,7 +629,7 @@ public class StatsCollectorIT extends BaseUniqueNamesOwnClusterIT {
             int startIndex = r.nextInt(strings.length);
             int endIndex = r.nextInt(strings.length - startIndex) + startIndex;
             long rows = endIndex - startIndex;
-            long c2Bytes = rows * 35;
+            long c2Bytes = rows * (columnEncoded ? ( mutable ? 37 : 48 ) : 35);
             String physicalTableName = SchemaUtil.getPhysicalHBaseTableName(fullTableName, userTableNamespaceMapped, PTableType.TABLE).getString();
             rs = conn.createStatement().executeQuery(
                     "SELECT COLUMN_FAMILY,SUM(GUIDE_POSTS_ROW_COUNT),SUM(GUIDE_POSTS_WIDTH) from SYSTEM.STATS where PHYSICAL_NAME = '"

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
index bb13f1b..a37903f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsIT.java
@@ -22,29 +22,38 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.IOException;
 import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
-import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Properties;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.client.HTable;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.expression.KeyValueColumnExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.tuple.ResultTuple;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
 
 /**
  * Tests to demonstrate and verify the STORE_NULLS option on a table,
@@ -52,74 +61,104 @@ import org.junit.Test;
  * functionality allows having row-level versioning (similar to how KEEP_DELETED_CELLS works), but
  * also allows permanently deleting a row.
  */
+@RunWith(Parameterized.class)
 public class StoreNullsIT extends ParallelStatsDisabledIT {
-    private static final Log LOG = LogFactory.getLog(StoreNullsIT.class);
     
-    private String WITH_NULLS;
-    private String WITHOUT_NULLS;
-    private String IMMUTABLE_WITH_NULLS;
-    private String IMMUTABLE_WITHOUT_NULLS;
-    private Connection conn;
-    private Statement stmt;
-
-    @Before
-    public void setUp() throws SQLException {
-        WITH_NULLS = generateUniqueName();
-        WITHOUT_NULLS = generateUniqueName();
-        IMMUTABLE_WITH_NULLS = generateUniqueName();
-        IMMUTABLE_WITHOUT_NULLS = generateUniqueName();
-        conn = DriverManager.getConnection(getUrl());
-        conn.setAutoCommit(true);
-
-        stmt = conn.createStatement();
-        stmt.execute("CREATE TABLE " + WITH_NULLS + " (" +
-                        "id SMALLINT NOT NULL PRIMARY KEY, " +
-                        "name VARCHAR) " +
-                "STORE_NULLS = true, VERSIONS = 1000, KEEP_DELETED_CELLS = false");
-        stmt.execute("CREATE TABLE " + WITHOUT_NULLS + " (" +
-                        "id SMALLINT NOT NULL PRIMARY KEY, " +
-                        "name VARCHAR) " +
-                "VERSIONS = 1000, KEEP_DELETED_CELLS = false");
-        stmt.execute("CREATE TABLE " + IMMUTABLE_WITH_NULLS + " ("
-                + "id SMALLINT NOT NULL PRIMARY KEY, name VARCHAR) "
-                + "STORE_NULLS = true, VERSIONS = 1, KEEP_DELETED_CELLS = false, IMMUTABLE_ROWS=true");
-        stmt.execute("CREATE TABLE " + IMMUTABLE_WITHOUT_NULLS + " ("
-                + "id SMALLINT NOT NULL PRIMARY KEY, name VARCHAR) "
-                + "VERSIONS = 1, KEEP_DELETED_CELLS = false, IMMUTABLE_ROWS=true");
+    private final boolean mutable;
+    private final boolean columnEncoded;
+    private final boolean storeNulls;
+    private final String ddlFormat;
+    
+    private String dataTableName;
+    
+    public StoreNullsIT(boolean mutable, boolean columnEncoded, boolean storeNulls) {
+        this.mutable = mutable;
+        this.columnEncoded = columnEncoded;
+        this.storeNulls = storeNulls;
+        
+        StringBuilder sb = new StringBuilder("CREATE TABLE %s (id SMALLINT NOT NULL PRIMARY KEY, name VARCHAR) VERSIONS = 1000, KEEP_DELETED_CELLS = false ");
+        if (!columnEncoded) {
+            sb.append(",").append("COLUMN_ENCODED_BYTES=0");
+        }
+        if (!mutable) {
+            sb.append(",").append("IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                sb.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
+        if (storeNulls) {
+            sb.append(",").append("STORE_NULLS=true");
+        }
+        this.ddlFormat = sb.toString();
     }
-
-    @After
-    public void tearDown() throws SQLException {
-        stmt.close();
-        conn.close();
+    
+    @Parameters(name="StoreNullsIT_mutable={0}, columnEncoded={1}, storeNulls={2}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] { 
+                { false, false, false }, { false, false, true },
+                { false, true, false }, { false, true, true },
+                { true, false, false }, { true, false, true },
+                { true, true, false }, { true, true, true }});
+    }
+    
+    
+    @Before
+    public void setupTableNames() throws Exception {
+        dataTableName = generateUniqueName();
     }
 
     @Test
-    public void testStoringNulls() throws SQLException, InterruptedException, IOException {
-        stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITH_NULLS + " VALUES (1, 'v1')");
-        stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITHOUT_NULLS + " VALUES (1, 'v1')");
-        stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITH_NULLS + " VALUES (2, null)");
-        stmt.executeUpdate("UPSERT INTO " + IMMUTABLE_WITHOUT_NULLS + " VALUES (2, null)");
-
-        ensureNullsNotStored(IMMUTABLE_WITH_NULLS);
-        ensureNullsNotStored(IMMUTABLE_WITHOUT_NULLS);
+    public void testStoringNullsForImmutableTables() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            conn.setAutoCommit(true);
+            stmt.execute(String.format(ddlFormat, dataTableName));
+            stmt.executeUpdate("UPSERT INTO " + dataTableName + " VALUES (1, 'v1')");
+            stmt.executeUpdate("UPSERT INTO " + dataTableName + " VALUES (2, null)");
+            TestUtil.doMajorCompaction(conn, dataTableName);
+            ensureNullsStoredCorrectly(conn);
+        }
     }
 
-    private void ensureNullsNotStored(String tableName) throws IOException {
-        tableName = SchemaUtil.normalizeIdentifier(tableName);
-        HTable htable = new HTable(getUtility().getConfiguration(), tableName);
+    private void ensureNullsStoredCorrectly(Connection conn) throws Exception {
+        ResultSet rs1 = conn.createStatement().executeQuery("SELECT NAME FROM "+dataTableName);
+        rs1.next();
+        assertEquals("v1", rs1.getString(1));
+        rs1.next();
+        assertNull(rs1.getString(1));
+        rs1.next();
+        
+        HTable htable = new HTable(getUtility().getConfiguration(), dataTableName);
         Scan s = new Scan();
         s.setRaw(true);
         ResultScanner scanner = htable.getScanner(s);
         // first row has a value for name
         Result rs = scanner.next();
-        assertTrue(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("NAME")));
-        assertTrue(rs.size() == 2);
-        // 2nd row has not
+        PTable table = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, dataTableName));
+        PColumn nameColumn = table.getColumnForColumnName("NAME");
+        byte[] qualifier = table.getImmutableStorageScheme()== ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS ? QueryConstants.SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES : nameColumn.getColumnQualifierBytes();
+        assertTrue(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, qualifier));
+        assertTrue(rs.size() == 2); // 2 because it also includes the empty key value column
+        KeyValueColumnExpression colExpression = table.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS ? new SingleCellColumnExpression(nameColumn, "NAME", table.getEncodingScheme()) : new KeyValueColumnExpression(nameColumn);
+        ImmutableBytesPtr ptr = new ImmutableBytesPtr();
+        colExpression.evaluate(new ResultTuple(rs), ptr);
+        assertEquals(new ImmutableBytesPtr(PVarchar.INSTANCE.toBytes("v1")), ptr);
         rs = scanner.next();
-        assertFalse(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, Bytes.toBytes("NAME")));
-        // and no delete marker either
-        assertTrue(rs.size() == 1);
+        
+        if ( !mutable && !columnEncoded // we don't issue a put with empty value for immutable tables with cols stored per key value
+                || (mutable && !storeNulls)) { // for this case we use a delete to represent the null
+            assertFalse(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, qualifier));
+            assertEquals(1, rs.size());
+        }
+        else { 
+            assertTrue(rs.containsColumn(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, qualifier));
+            assertEquals(2, rs.size()); 
+        }
+        // assert null stored correctly 
+        ptr = new ImmutableBytesPtr();
+        if (colExpression.evaluate(new ResultTuple(rs), ptr)) {
+            assertEquals(new ImmutableBytesPtr(ByteUtil.EMPTY_BYTE_ARRAY), ptr);
+        }
         assertNull(scanner.next());
         scanner.close();
         htable.close();
@@ -127,93 +166,80 @@ public class StoreNullsIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testQueryingHistory() throws Exception {
-        stmt.executeUpdate("UPSERT INTO " + WITH_NULLS + " VALUES (1, 'v1')");
-        stmt.executeUpdate("UPSERT INTO " + WITHOUT_NULLS + " VALUES (1, 'v1')");
-
-        Thread.sleep(10L);
-        long afterFirstInsert = System.currentTimeMillis();
-        Thread.sleep(10L);
-
-        stmt.executeUpdate("UPSERT INTO " + WITH_NULLS + " VALUES (1, null)");
-        stmt.executeUpdate("UPSERT INTO " + WITHOUT_NULLS + " VALUES (1, null)");
-        Thread.sleep(10L);
-
-        TestUtil.doMajorCompaction(conn, WITH_NULLS);
-        TestUtil.doMajorCompaction(conn, WITHOUT_NULLS);
-
-        Properties historicalProps = new Properties();
-        historicalProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            conn.setAutoCommit(true);
+            stmt.execute(String.format(ddlFormat, dataTableName));
+            stmt.executeUpdate("UPSERT INTO " + dataTableName + " VALUES (1, 'v1')");
+            Thread.sleep(10L);
+            long afterFirstInsert = System.currentTimeMillis();
+            Thread.sleep(10L);
+            
+            stmt.executeUpdate("UPSERT INTO " + dataTableName + " VALUES (1, null)");
+            Thread.sleep(10L);
+            
+            TestUtil.doMajorCompaction(conn, dataTableName);
+            
+            Properties historicalProps = new Properties();
+            historicalProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
                 Long.toString(afterFirstInsert));
-        Connection historicalConn = DriverManager.getConnection(getUrl(), historicalProps);
-        Statement historicalStmt = historicalConn.createStatement();
-
-        ResultSet rs = historicalStmt.executeQuery(
-            "SELECT name FROM " + WITH_NULLS + " WHERE id = 1");
-        assertTrue(rs.next());
-        assertEquals("v1", rs.getString(1));
-        rs.close();
-
-        // The single null wipes out all history for a field if STORE_NULLS is not enabled
-        rs = historicalStmt.executeQuery("SELECT name FROM " + WITHOUT_NULLS + " WHERE id = 1");
-        assertTrue(rs.next());
-        assertNull(rs.getString(1));
-        rs.close();
+            Connection historicalConn = DriverManager.getConnection(getUrl(), historicalProps);
+            Statement historicalStmt = historicalConn.createStatement();
+            ResultSet rs = historicalStmt.executeQuery( "SELECT name FROM " + dataTableName + " WHERE id = 1");
+            
+            if (storeNulls || !mutable) { // store nulls is set to true if the table is immutable
+                assertTrue(rs.next());
+                assertEquals("v1", rs.getString(1));
+                rs.close();
+            } 
+            else {
+                // The single null wipes out all history for a field if STORE_NULLS is not enabled
+                assertTrue(rs.next());
+                assertNull(rs.getString(1));
+            }
+            
+            rs.close();
+            historicalStmt.close();
+            historicalConn.close();
+        }
 
-        historicalStmt.close();
-        historicalConn.close();
     }
 
     // Row deletes should work in the same way regardless of what STORE_NULLS is set to
     @Test
     public void testDeletes() throws Exception {
-        stmt.executeUpdate("UPSERT INTO " + WITH_NULLS + " VALUES (1, 'v1')");
-        stmt.executeUpdate("UPSERT INTO " + WITHOUT_NULLS + " VALUES (1, 'v1')");
-
-        Thread.sleep(10L);
-        long afterFirstInsert = System.currentTimeMillis();
-        Thread.sleep(10L);
-
-        stmt.executeUpdate("DELETE FROM " + WITH_NULLS + " WHERE id = 1");
-        stmt.executeUpdate("DELETE FROM " + WITHOUT_NULLS + " WHERE id = 1");
-        Thread.sleep(10L);
-
-        TestUtil.doMajorCompaction(conn, WITH_NULLS);
-        TestUtil.doMajorCompaction(conn, WITHOUT_NULLS);
-
-        Properties historicalProps = new Properties();
-        historicalProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
-                Long.toString(afterFirstInsert));
-        Connection historicalConn = DriverManager.getConnection(getUrl(), historicalProps);
-        Statement historicalStmt = historicalConn.createStatement();
-
-        // The row should be completely gone for both tables now
-
-        ResultSet rs = historicalStmt.executeQuery(
-            "SELECT name FROM " + WITH_NULLS + " WHERE id = 1");
-        assertFalse(rs.next());
-        rs.close();
-
-        rs = historicalStmt.executeQuery("SELECT name FROM " + WITHOUT_NULLS + " WHERE id = 1");
-        assertFalse(rs.next());
-        rs.close();
-    }
-
-    @Test
-    public void testSetStoreNullsDefaultViaConfig() throws SQLException {
-        Properties props = new Properties();
-        props.setProperty(QueryServices.DEFAULT_STORE_NULLS_ATTRIB, "true");
-        Connection storeNullsConn = DriverManager.getConnection(getUrl(), props);
-
-        Statement stmt = storeNullsConn.createStatement();
-        stmt.execute("CREATE TABLE with_nulls_default (" +
-                "id smallint primary key," +
-                "name varchar)");
-
-        ResultSet rs = stmt.executeQuery("SELECT store_nulls FROM SYSTEM.CATALOG " +
-                "WHERE table_name = 'WITH_NULLS_DEFAULT' AND store_nulls is not null");
-        assertTrue(rs.next());
-        assertTrue(rs.getBoolean(1));
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            conn.setAutoCommit(true);
+            stmt.execute(String.format(ddlFormat, dataTableName));
+            stmt.executeUpdate("UPSERT INTO " + dataTableName + " VALUES (1, 'v1')");
+    
+            Thread.sleep(10L);
+            long afterFirstInsert = System.currentTimeMillis();
+            Thread.sleep(10L);
+    
+            stmt.executeUpdate("DELETE FROM " + dataTableName + " WHERE id = 1");
+            Thread.sleep(10L);
+    
+            TestUtil.doMajorCompaction(conn, dataTableName);
+    
+            Properties historicalProps = new Properties();
+            historicalProps.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB,
+                    Long.toString(afterFirstInsert));
+            Connection historicalConn = DriverManager.getConnection(getUrl(), historicalProps);
+            Statement historicalStmt = historicalConn.createStatement();
+    
+            // The row should be completely gone for both tables now
+    
+            ResultSet rs = historicalStmt.executeQuery(
+                "SELECT name FROM " + dataTableName + " WHERE id = 1");
+            assertFalse(rs.next());
+            rs.close();
+    
+            rs = historicalStmt.executeQuery("SELECT name FROM " + dataTableName + " WHERE id = 1");
+            assertFalse(rs.next());
+            rs.close();
+        }
     }
 
-
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsPropIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsPropIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsPropIT.java
new file mode 100644
index 0000000..26ff629
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/StoreNullsPropIT.java
@@ -0,0 +1,51 @@
+/*
+ * 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.end2end;
+
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.query.QueryServices;
+import org.junit.Test;
+
+public class StoreNullsPropIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testSetStoreNullsDefaultViaConfig() throws SQLException {
+        Properties props = new Properties();
+        props.setProperty(QueryServices.DEFAULT_STORE_NULLS_ATTRIB, "true");
+        Connection storeNullsConn = DriverManager.getConnection(getUrl(), props);
+
+        Statement stmt = storeNullsConn.createStatement();
+        stmt.execute("CREATE TABLE with_nulls_default (" +
+                "id smallint primary key," +
+                "name varchar)");
+
+        ResultSet rs = stmt.executeQuery("SELECT store_nulls FROM SYSTEM.CATALOG " +
+                "WHERE table_name = 'WITH_NULLS_DEFAULT' AND store_nulls is not null");
+        assertTrue(rs.next());
+        assertTrue(rs.getBoolean(1));
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
index 6b394c1..f9ef0c2 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/SysTableNamespaceMappedStatsCollectorIT.java
@@ -27,8 +27,8 @@ import com.google.common.collect.Maps;
 
 public class SysTableNamespaceMappedStatsCollectorIT extends StatsCollectorIT {
     
-    public SysTableNamespaceMappedStatsCollectorIT(boolean transactional, boolean userTableNamespaceMapped) {
-        super(transactional, userTableNamespaceMapped);
+    public SysTableNamespaceMappedStatsCollectorIT(boolean mutable, boolean transactional, boolean userTableNamespaceMapped, boolean columnEncoded) {
+        super(mutable, transactional, userTableNamespaceMapped, columnEncoded);
     }
     
     @BeforeClass

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
index ca1cd86..39e8cb6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TopNIT.java
@@ -50,7 +50,7 @@ public class TopNIT extends BaseClientManagedTimeIT {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
 
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT entity_id FROM aTable ORDER BY b_string, entity_id LIMIT 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -80,7 +80,7 @@ public class TopNIT extends BaseClientManagedTimeIT {
     public void testDescMultiOrderByExpr() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "SELECT entity_id FROM aTable ORDER BY b_string || entity_id desc LIMIT 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -119,7 +119,7 @@ public class TopNIT extends BaseClientManagedTimeIT {
     private void testTopNDelete(boolean autoCommit) throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         String query = "DELETE FROM aTable ORDER BY b_string, entity_id LIMIT 5";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2


[09/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
index 5ee1dfb..3b8984a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ServerCachingProtos.java
@@ -649,6 +649,4947 @@ public final class ServerCachingProtos {
     // @@protoc_insertion_point(class_scope:ImmutableBytesWritable)
   }
 
+  public interface ColumnReferenceOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required bytes family = 1;
+    /**
+     * <code>required bytes family = 1;</code>
+     */
+    boolean hasFamily();
+    /**
+     * <code>required bytes family = 1;</code>
+     */
+    com.google.protobuf.ByteString getFamily();
+
+    // required bytes qualifier = 2;
+    /**
+     * <code>required bytes qualifier = 2;</code>
+     */
+    boolean hasQualifier();
+    /**
+     * <code>required bytes qualifier = 2;</code>
+     */
+    com.google.protobuf.ByteString getQualifier();
+  }
+  /**
+   * Protobuf type {@code ColumnReference}
+   */
+  public static final class ColumnReference extends
+      com.google.protobuf.GeneratedMessage
+      implements ColumnReferenceOrBuilder {
+    // Use ColumnReference.newBuilder() to construct.
+    private ColumnReference(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ColumnReference(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ColumnReference defaultInstance;
+    public static ColumnReference getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ColumnReference getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ColumnReference(
+        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;
+              family_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              qualifier_ = input.readBytes();
+              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.ServerCachingProtos.internal_static_ColumnReference_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_ColumnReference_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.class, org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ColumnReference> PARSER =
+        new com.google.protobuf.AbstractParser<ColumnReference>() {
+      public ColumnReference parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ColumnReference(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ColumnReference> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required bytes family = 1;
+    public static final int FAMILY_FIELD_NUMBER = 1;
+    private com.google.protobuf.ByteString family_;
+    /**
+     * <code>required bytes family = 1;</code>
+     */
+    public boolean hasFamily() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required bytes family = 1;</code>
+     */
+    public com.google.protobuf.ByteString getFamily() {
+      return family_;
+    }
+
+    // required bytes qualifier = 2;
+    public static final int QUALIFIER_FIELD_NUMBER = 2;
+    private com.google.protobuf.ByteString qualifier_;
+    /**
+     * <code>required bytes qualifier = 2;</code>
+     */
+    public boolean hasQualifier() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bytes qualifier = 2;</code>
+     */
+    public com.google.protobuf.ByteString getQualifier() {
+      return qualifier_;
+    }
+
+    private void initFields() {
+      family_ = com.google.protobuf.ByteString.EMPTY;
+      qualifier_ = com.google.protobuf.ByteString.EMPTY;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasFamily()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasQualifier()) {
+        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, family_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, qualifier_);
+      }
+      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, family_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, qualifier_);
+      }
+      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.ServerCachingProtos.ColumnReference)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference other = (org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference) obj;
+
+      boolean result = true;
+      result = result && (hasFamily() == other.hasFamily());
+      if (hasFamily()) {
+        result = result && getFamily()
+            .equals(other.getFamily());
+      }
+      result = result && (hasQualifier() == other.hasQualifier());
+      if (hasQualifier()) {
+        result = result && getQualifier()
+            .equals(other.getQualifier());
+      }
+      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 (hasFamily()) {
+        hash = (37 * hash) + FAMILY_FIELD_NUMBER;
+        hash = (53 * hash) + getFamily().hashCode();
+      }
+      if (hasQualifier()) {
+        hash = (37 * hash) + QUALIFIER_FIELD_NUMBER;
+        hash = (53 * hash) + getQualifier().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference 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.ServerCachingProtos.ColumnReference parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference 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.ServerCachingProtos.ColumnReference parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference 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.ServerCachingProtos.ColumnReference parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference 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.ServerCachingProtos.ColumnReference parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference 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.ServerCachingProtos.ColumnReference 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 ColumnReference}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_ColumnReference_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_ColumnReference_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.class, org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.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();
+        family_ = com.google.protobuf.ByteString.EMPTY;
+        bitField0_ = (bitField0_ & ~0x00000001);
+        qualifier_ = com.google.protobuf.ByteString.EMPTY;
+        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.ServerCachingProtos.internal_static_ColumnReference_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference build() {
+        org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference buildPartial() {
+        org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference result = new org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.family_ = family_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.qualifier_ = qualifier_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference other) {
+        if (other == org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.getDefaultInstance()) return this;
+        if (other.hasFamily()) {
+          setFamily(other.getFamily());
+        }
+        if (other.hasQualifier()) {
+          setQualifier(other.getQualifier());
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasFamily()) {
+          
+          return false;
+        }
+        if (!hasQualifier()) {
+          
+          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.ServerCachingProtos.ColumnReference parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // required bytes family = 1;
+      private com.google.protobuf.ByteString family_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes family = 1;</code>
+       */
+      public boolean hasFamily() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>required bytes family = 1;</code>
+       */
+      public com.google.protobuf.ByteString getFamily() {
+        return family_;
+      }
+      /**
+       * <code>required bytes family = 1;</code>
+       */
+      public Builder setFamily(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        family_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes family = 1;</code>
+       */
+      public Builder clearFamily() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        family_ = getDefaultInstance().getFamily();
+        onChanged();
+        return this;
+      }
+
+      // required bytes qualifier = 2;
+      private com.google.protobuf.ByteString qualifier_ = com.google.protobuf.ByteString.EMPTY;
+      /**
+       * <code>required bytes qualifier = 2;</code>
+       */
+      public boolean hasQualifier() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required bytes qualifier = 2;</code>
+       */
+      public com.google.protobuf.ByteString getQualifier() {
+        return qualifier_;
+      }
+      /**
+       * <code>required bytes qualifier = 2;</code>
+       */
+      public Builder setQualifier(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        qualifier_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required bytes qualifier = 2;</code>
+       */
+      public Builder clearQualifier() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        qualifier_ = getDefaultInstance().getQualifier();
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ColumnReference)
+    }
+
+    static {
+      defaultInstance = new ColumnReference(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ColumnReference)
+  }
+
+  public interface ColumnInfoOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string familyName = 1;
+    /**
+     * <code>optional string familyName = 1;</code>
+     */
+    boolean hasFamilyName();
+    /**
+     * <code>optional string familyName = 1;</code>
+     */
+    java.lang.String getFamilyName();
+    /**
+     * <code>optional string familyName = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getFamilyNameBytes();
+
+    // required string columnName = 2;
+    /**
+     * <code>required string columnName = 2;</code>
+     */
+    boolean hasColumnName();
+    /**
+     * <code>required string columnName = 2;</code>
+     */
+    java.lang.String getColumnName();
+    /**
+     * <code>required string columnName = 2;</code>
+     */
+    com.google.protobuf.ByteString
+        getColumnNameBytes();
+  }
+  /**
+   * Protobuf type {@code ColumnInfo}
+   */
+  public static final class ColumnInfo extends
+      com.google.protobuf.GeneratedMessage
+      implements ColumnInfoOrBuilder {
+    // Use ColumnInfo.newBuilder() to construct.
+    private ColumnInfo(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private ColumnInfo(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final ColumnInfo defaultInstance;
+    public static ColumnInfo getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public ColumnInfo getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private ColumnInfo(
+        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;
+              familyName_ = input.readBytes();
+              break;
+            }
+            case 18: {
+              bitField0_ |= 0x00000002;
+              columnName_ = input.readBytes();
+              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.ServerCachingProtos.internal_static_ColumnInfo_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_ColumnInfo_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.class, org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<ColumnInfo> PARSER =
+        new com.google.protobuf.AbstractParser<ColumnInfo>() {
+      public ColumnInfo parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new ColumnInfo(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<ColumnInfo> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional string familyName = 1;
+    public static final int FAMILYNAME_FIELD_NUMBER = 1;
+    private java.lang.Object familyName_;
+    /**
+     * <code>optional string familyName = 1;</code>
+     */
+    public boolean hasFamilyName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string familyName = 1;</code>
+     */
+    public java.lang.String getFamilyName() {
+      java.lang.Object ref = familyName_;
+      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()) {
+          familyName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string familyName = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getFamilyNameBytes() {
+      java.lang.Object ref = familyName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        familyName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    // required string columnName = 2;
+    public static final int COLUMNNAME_FIELD_NUMBER = 2;
+    private java.lang.Object columnName_;
+    /**
+     * <code>required string columnName = 2;</code>
+     */
+    public boolean hasColumnName() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required string columnName = 2;</code>
+     */
+    public java.lang.String getColumnName() {
+      java.lang.Object ref = columnName_;
+      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()) {
+          columnName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>required string columnName = 2;</code>
+     */
+    public com.google.protobuf.ByteString
+        getColumnNameBytes() {
+      java.lang.Object ref = columnName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        columnName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      familyName_ = "";
+      columnName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasColumnName()) {
+        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, getFamilyNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBytes(2, getColumnNameBytes());
+      }
+      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, getFamilyNameBytes());
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(2, getColumnNameBytes());
+      }
+      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.ServerCachingProtos.ColumnInfo)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo other = (org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo) obj;
+
+      boolean result = true;
+      result = result && (hasFamilyName() == other.hasFamilyName());
+      if (hasFamilyName()) {
+        result = result && getFamilyName()
+            .equals(other.getFamilyName());
+      }
+      result = result && (hasColumnName() == other.hasColumnName());
+      if (hasColumnName()) {
+        result = result && getColumnName()
+            .equals(other.getColumnName());
+      }
+      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 (hasFamilyName()) {
+        hash = (37 * hash) + FAMILYNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getFamilyName().hashCode();
+      }
+      if (hasColumnName()) {
+        hash = (37 * hash) + COLUMNNAME_FIELD_NUMBER;
+        hash = (53 * hash) + getColumnName().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo 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.ServerCachingProtos.ColumnInfo parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo 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.ServerCachingProtos.ColumnInfo parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo 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.ServerCachingProtos.ColumnInfo parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo 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.ServerCachingProtos.ColumnInfo parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo 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.ServerCachingProtos.ColumnInfo 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 ColumnInfo}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfoOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_ColumnInfo_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_ColumnInfo_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.class, org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.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();
+        familyName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        columnName_ = "";
+        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.ServerCachingProtos.internal_static_ColumnInfo_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo build() {
+        org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo buildPartial() {
+        org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo result = new org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.familyName_ = familyName_;
+        if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        result.columnName_ = columnName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo other) {
+        if (other == org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.getDefaultInstance()) return this;
+        if (other.hasFamilyName()) {
+          bitField0_ |= 0x00000001;
+          familyName_ = other.familyName_;
+          onChanged();
+        }
+        if (other.hasColumnName()) {
+          bitField0_ |= 0x00000002;
+          columnName_ = other.columnName_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        if (!hasColumnName()) {
+          
+          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.ServerCachingProtos.ColumnInfo parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional string familyName = 1;
+      private java.lang.Object familyName_ = "";
+      /**
+       * <code>optional string familyName = 1;</code>
+       */
+      public boolean hasFamilyName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string familyName = 1;</code>
+       */
+      public java.lang.String getFamilyName() {
+        java.lang.Object ref = familyName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          familyName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string familyName = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getFamilyNameBytes() {
+        java.lang.Object ref = familyName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          familyName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string familyName = 1;</code>
+       */
+      public Builder setFamilyName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        familyName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string familyName = 1;</code>
+       */
+      public Builder clearFamilyName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        familyName_ = getDefaultInstance().getFamilyName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string familyName = 1;</code>
+       */
+      public Builder setFamilyNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        familyName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // required string columnName = 2;
+      private java.lang.Object columnName_ = "";
+      /**
+       * <code>required string columnName = 2;</code>
+       */
+      public boolean hasColumnName() {
+        return ((bitField0_ & 0x00000002) == 0x00000002);
+      }
+      /**
+       * <code>required string columnName = 2;</code>
+       */
+      public java.lang.String getColumnName() {
+        java.lang.Object ref = columnName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          columnName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>required string columnName = 2;</code>
+       */
+      public com.google.protobuf.ByteString
+          getColumnNameBytes() {
+        java.lang.Object ref = columnName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          columnName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>required string columnName = 2;</code>
+       */
+      public Builder setColumnName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        columnName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string columnName = 2;</code>
+       */
+      public Builder clearColumnName() {
+        bitField0_ = (bitField0_ & ~0x00000002);
+        columnName_ = getDefaultInstance().getColumnName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>required string columnName = 2;</code>
+       */
+      public Builder setColumnNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000002;
+        columnName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:ColumnInfo)
+    }
+
+    static {
+      defaultInstance = new ColumnInfo(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:ColumnInfo)
+  }
+
+  public interface IndexMaintainerOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // required int32 saltBuckets = 1;
+    /**
+     * <code>required int32 saltBuckets = 1;</code>
+     */
+    boolean hasSaltBuckets();
+    /**
+     * <code>required int32 saltBuckets = 1;</code>
+     */
+    int getSaltBuckets();
+
+    // required bool isMultiTenant = 2;
+    /**
+     * <code>required bool isMultiTenant = 2;</code>
+     */
+    boolean hasIsMultiTenant();
+    /**
+     * <code>required bool isMultiTenant = 2;</code>
+     */
+    boolean getIsMultiTenant();
+
+    // optional bytes viewIndexId = 3;
+    /**
+     * <code>optional bytes viewIndexId = 3;</code>
+     */
+    boolean hasViewIndexId();
+    /**
+     * <code>optional bytes viewIndexId = 3;</code>
+     */
+    com.google.protobuf.ByteString getViewIndexId();
+
+    // repeated .ColumnReference indexedColumns = 4;
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> 
+        getIndexedColumnsList();
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getIndexedColumns(int index);
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    int getIndexedColumnsCount();
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder> 
+        getIndexedColumnsOrBuilderList();
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder getIndexedColumnsOrBuilder(
+        int index);
+
+    // repeated int32 indexedColumnTypeOrdinal = 5;
+    /**
+     * <code>repeated int32 indexedColumnTypeOrdinal = 5;</code>
+     */
+    java.util.List<java.lang.Integer> getIndexedColumnTypeOrdinalList();
+    /**
+     * <code>repeated int32 indexedColumnTypeOrdinal = 5;</code>
+     */
+    int getIndexedColumnTypeOrdinalCount();
+    /**
+     * <code>repeated int32 indexedColumnTypeOrdinal = 5;</code>
+     */
+    int getIndexedColumnTypeOrdinal(int index);
+
+    // repeated .ColumnReference dataTableColRefForCoveredColumns = 6;
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> 
+        getDataTableColRefForCoveredColumnsList();
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getDataTableColRefForCoveredColumns(int index);
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    int getDataTableColRefForCoveredColumnsCount();
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder> 
+        getDataTableColRefForCoveredColumnsOrBuilderList();
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder getDataTableColRefForCoveredColumnsOrBuilder(
+        int index);
+
+    // repeated .ColumnReference indexTableColRefForCoveredColumns = 7;
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> 
+        getIndexTableColRefForCoveredColumnsList();
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getIndexTableColRefForCoveredColumns(int index);
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    int getIndexTableColRefForCoveredColumnsCount();
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder> 
+        getIndexTableColRefForCoveredColumnsOrBuilderList();
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder getIndexTableColRefForCoveredColumnsOrBuilder(
+        int index);
+
+    // required bool isLocalIndex = 8;
+    /**
+     * <code>required bool isLocalIndex = 8;</code>
+     */
+    boolean hasIsLocalIndex();
+    /**
+     * <code>required bool isLocalIndex = 8;</code>
+     */
+    boolean getIsLocalIndex();
+
+    // required bytes indexTableName = 9;
+    /**
+     * <code>required bytes indexTableName = 9;</code>
+     */
+    boolean hasIndexTableName();
+    /**
+     * <code>required bytes indexTableName = 9;</code>
+     */
+    com.google.protobuf.ByteString getIndexTableName();
+
+    // required bool rowKeyOrderOptimizable = 10;
+    /**
+     * <code>required bool rowKeyOrderOptimizable = 10;</code>
+     */
+    boolean hasRowKeyOrderOptimizable();
+    /**
+     * <code>required bool rowKeyOrderOptimizable = 10;</code>
+     */
+    boolean getRowKeyOrderOptimizable();
+
+    // required bytes dataTableEmptyKeyValueColFamily = 11;
+    /**
+     * <code>required bytes dataTableEmptyKeyValueColFamily = 11;</code>
+     */
+    boolean hasDataTableEmptyKeyValueColFamily();
+    /**
+     * <code>required bytes dataTableEmptyKeyValueColFamily = 11;</code>
+     */
+    com.google.protobuf.ByteString getDataTableEmptyKeyValueColFamily();
+
+    // required .ImmutableBytesWritable emptyKeyValueColFamily = 12;
+    /**
+     * <code>required .ImmutableBytesWritable emptyKeyValueColFamily = 12;</code>
+     */
+    boolean hasEmptyKeyValueColFamily();
+    /**
+     * <code>required .ImmutableBytesWritable emptyKeyValueColFamily = 12;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritable getEmptyKeyValueColFamily();
+    /**
+     * <code>required .ImmutableBytesWritable emptyKeyValueColFamily = 12;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritableOrBuilder getEmptyKeyValueColFamilyOrBuilder();
+
+    // optional bytes indexedExpressions = 13;
+    /**
+     * <code>optional bytes indexedExpressions = 13;</code>
+     */
+    boolean hasIndexedExpressions();
+    /**
+     * <code>optional bytes indexedExpressions = 13;</code>
+     */
+    com.google.protobuf.ByteString getIndexedExpressions();
+
+    // required bytes rowKeyMetadata = 14;
+    /**
+     * <code>required bytes rowKeyMetadata = 14;</code>
+     */
+    boolean hasRowKeyMetadata();
+    /**
+     * <code>required bytes rowKeyMetadata = 14;</code>
+     */
+    com.google.protobuf.ByteString getRowKeyMetadata();
+
+    // required int32 numDataTableColFamilies = 15;
+    /**
+     * <code>required int32 numDataTableColFamilies = 15;</code>
+     */
+    boolean hasNumDataTableColFamilies();
+    /**
+     * <code>required int32 numDataTableColFamilies = 15;</code>
+     */
+    int getNumDataTableColFamilies();
+
+    // required bool indexWalDisabled = 16;
+    /**
+     * <code>required bool indexWalDisabled = 16;</code>
+     */
+    boolean hasIndexWalDisabled();
+    /**
+     * <code>required bool indexWalDisabled = 16;</code>
+     */
+    boolean getIndexWalDisabled();
+
+    // required int32 indexRowKeyByteSize = 17;
+    /**
+     * <code>required int32 indexRowKeyByteSize = 17;</code>
+     */
+    boolean hasIndexRowKeyByteSize();
+    /**
+     * <code>required int32 indexRowKeyByteSize = 17;</code>
+     */
+    int getIndexRowKeyByteSize();
+
+    // required bool immutable = 18;
+    /**
+     * <code>required bool immutable = 18;</code>
+     */
+    boolean hasImmutable();
+    /**
+     * <code>required bool immutable = 18;</code>
+     */
+    boolean getImmutable();
+
+    // repeated .ColumnInfo indexedColumnInfo = 19;
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo> 
+        getIndexedColumnInfoList();
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo getIndexedColumnInfo(int index);
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    int getIndexedColumnInfoCount();
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfoOrBuilder> 
+        getIndexedColumnInfoOrBuilderList();
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfoOrBuilder getIndexedColumnInfoOrBuilder(
+        int index);
+
+    // required int32 encodingScheme = 20;
+    /**
+     * <code>required int32 encodingScheme = 20;</code>
+     */
+    boolean hasEncodingScheme();
+    /**
+     * <code>required int32 encodingScheme = 20;</code>
+     */
+    int getEncodingScheme();
+
+    // required int32 immutableStorageScheme = 21;
+    /**
+     * <code>required int32 immutableStorageScheme = 21;</code>
+     */
+    boolean hasImmutableStorageScheme();
+    /**
+     * <code>required int32 immutableStorageScheme = 21;</code>
+     */
+    int getImmutableStorageScheme();
+  }
+  /**
+   * Protobuf type {@code IndexMaintainer}
+   */
+  public static final class IndexMaintainer extends
+      com.google.protobuf.GeneratedMessage
+      implements IndexMaintainerOrBuilder {
+    // Use IndexMaintainer.newBuilder() to construct.
+    private IndexMaintainer(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private IndexMaintainer(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final IndexMaintainer defaultInstance;
+    public static IndexMaintainer getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public IndexMaintainer getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private IndexMaintainer(
+        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 8: {
+              bitField0_ |= 0x00000001;
+              saltBuckets_ = input.readInt32();
+              break;
+            }
+            case 16: {
+              bitField0_ |= 0x00000002;
+              isMultiTenant_ = input.readBool();
+              break;
+            }
+            case 26: {
+              bitField0_ |= 0x00000004;
+              viewIndexId_ = input.readBytes();
+              break;
+            }
+            case 34: {
+              if (!((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+                indexedColumns_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference>();
+                mutable_bitField0_ |= 0x00000008;
+              }
+              indexedColumns_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.PARSER, extensionRegistry));
+              break;
+            }
+            case 40: {
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+                indexedColumnTypeOrdinal_ = new java.util.ArrayList<java.lang.Integer>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              indexedColumnTypeOrdinal_.add(input.readInt32());
+              break;
+            }
+            case 42: {
+              int length = input.readRawVarint32();
+              int limit = input.pushLimit(length);
+              if (!((mutable_bitField0_ & 0x00000010) == 0x00000010) && input.getBytesUntilLimit() > 0) {
+                indexedColumnTypeOrdinal_ = new java.util.ArrayList<java.lang.Integer>();
+                mutable_bitField0_ |= 0x00000010;
+              }
+              while (input.getBytesUntilLimit() > 0) {
+                indexedColumnTypeOrdinal_.add(input.readInt32());
+              }
+              input.popLimit(limit);
+              break;
+            }
+            case 50: {
+              if (!((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+                dataTableColRefForCoveredColumns_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference>();
+                mutable_bitField0_ |= 0x00000020;
+              }
+              dataTableColRefForCoveredColumns_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.PARSER, extensionRegistry));
+              break;
+            }
+            case 58: {
+              if (!((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+                indexTableColRefForCoveredColumns_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference>();
+                mutable_bitField0_ |= 0x00000040;
+              }
+              indexTableColRefForCoveredColumns_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference.PARSER, extensionRegistry));
+              break;
+            }
+            case 64: {
+              bitField0_ |= 0x00000008;
+              isLocalIndex_ = input.readBool();
+              break;
+            }
+            case 74: {
+              bitField0_ |= 0x00000010;
+              indexTableName_ = input.readBytes();
+              break;
+            }
+            case 80: {
+              bitField0_ |= 0x00000020;
+              rowKeyOrderOptimizable_ = input.readBool();
+              break;
+            }
+            case 90: {
+              bitField0_ |= 0x00000040;
+              dataTableEmptyKeyValueColFamily_ = input.readBytes();
+              break;
+            }
+            case 98: {
+              org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritable.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000080) == 0x00000080)) {
+                subBuilder = emptyKeyValueColFamily_.toBuilder();
+              }
+              emptyKeyValueColFamily_ = input.readMessage(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritable.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(emptyKeyValueColFamily_);
+                emptyKeyValueColFamily_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000080;
+              break;
+            }
+            case 106: {
+              bitField0_ |= 0x00000100;
+              indexedExpressions_ = input.readBytes();
+              break;
+            }
+            case 114: {
+              bitField0_ |= 0x00000200;
+              rowKeyMetadata_ = input.readBytes();
+              break;
+            }
+            case 120: {
+              bitField0_ |= 0x00000400;
+              numDataTableColFamilies_ = input.readInt32();
+              break;
+            }
+            case 128: {
+              bitField0_ |= 0x00000800;
+              indexWalDisabled_ = input.readBool();
+              break;
+            }
+            case 136: {
+              bitField0_ |= 0x00001000;
+              indexRowKeyByteSize_ = input.readInt32();
+              break;
+            }
+            case 144: {
+              bitField0_ |= 0x00002000;
+              immutable_ = input.readBool();
+              break;
+            }
+            case 154: {
+              if (!((mutable_bitField0_ & 0x00040000) == 0x00040000)) {
+                indexedColumnInfo_ = new java.util.ArrayList<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo>();
+                mutable_bitField0_ |= 0x00040000;
+              }
+              indexedColumnInfo_.add(input.readMessage(org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo.PARSER, extensionRegistry));
+              break;
+            }
+            case 160: {
+              bitField0_ |= 0x00004000;
+              encodingScheme_ = input.readInt32();
+              break;
+            }
+            case 168: {
+              bitField0_ |= 0x00008000;
+              immutableStorageScheme_ = 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 {
+        if (((mutable_bitField0_ & 0x00000008) == 0x00000008)) {
+          indexedColumns_ = java.util.Collections.unmodifiableList(indexedColumns_);
+        }
+        if (((mutable_bitField0_ & 0x00000010) == 0x00000010)) {
+          indexedColumnTypeOrdinal_ = java.util.Collections.unmodifiableList(indexedColumnTypeOrdinal_);
+        }
+        if (((mutable_bitField0_ & 0x00000020) == 0x00000020)) {
+          dataTableColRefForCoveredColumns_ = java.util.Collections.unmodifiableList(dataTableColRefForCoveredColumns_);
+        }
+        if (((mutable_bitField0_ & 0x00000040) == 0x00000040)) {
+          indexTableColRefForCoveredColumns_ = java.util.Collections.unmodifiableList(indexTableColRefForCoveredColumns_);
+        }
+        if (((mutable_bitField0_ & 0x00040000) == 0x00040000)) {
+          indexedColumnInfo_ = java.util.Collections.unmodifiableList(indexedColumnInfo_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_IndexMaintainer_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.ServerCachingProtos.internal_static_IndexMaintainer_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.ServerCachingProtos.IndexMaintainer.class, org.apache.phoenix.coprocessor.generated.ServerCachingProtos.IndexMaintainer.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<IndexMaintainer> PARSER =
+        new com.google.protobuf.AbstractParser<IndexMaintainer>() {
+      public IndexMaintainer parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new IndexMaintainer(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<IndexMaintainer> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // required int32 saltBuckets = 1;
+    public static final int SALTBUCKETS_FIELD_NUMBER = 1;
+    private int saltBuckets_;
+    /**
+     * <code>required int32 saltBuckets = 1;</code>
+     */
+    public boolean hasSaltBuckets() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>required int32 saltBuckets = 1;</code>
+     */
+    public int getSaltBuckets() {
+      return saltBuckets_;
+    }
+
+    // required bool isMultiTenant = 2;
+    public static final int ISMULTITENANT_FIELD_NUMBER = 2;
+    private boolean isMultiTenant_;
+    /**
+     * <code>required bool isMultiTenant = 2;</code>
+     */
+    public boolean hasIsMultiTenant() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>required bool isMultiTenant = 2;</code>
+     */
+    public boolean getIsMultiTenant() {
+      return isMultiTenant_;
+    }
+
+    // optional bytes viewIndexId = 3;
+    public static final int VIEWINDEXID_FIELD_NUMBER = 3;
+    private com.google.protobuf.ByteString viewIndexId_;
+    /**
+     * <code>optional bytes viewIndexId = 3;</code>
+     */
+    public boolean hasViewIndexId() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional bytes viewIndexId = 3;</code>
+     */
+    public com.google.protobuf.ByteString getViewIndexId() {
+      return viewIndexId_;
+    }
+
+    // repeated .ColumnReference indexedColumns = 4;
+    public static final int INDEXEDCOLUMNS_FIELD_NUMBER = 4;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> indexedColumns_;
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> getIndexedColumnsList() {
+      return indexedColumns_;
+    }
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder> 
+        getIndexedColumnsOrBuilderList() {
+      return indexedColumns_;
+    }
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    public int getIndexedColumnsCount() {
+      return indexedColumns_.size();
+    }
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getIndexedColumns(int index) {
+      return indexedColumns_.get(index);
+    }
+    /**
+     * <code>repeated .ColumnReference indexedColumns = 4;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder getIndexedColumnsOrBuilder(
+        int index) {
+      return indexedColumns_.get(index);
+    }
+
+    // repeated int32 indexedColumnTypeOrdinal = 5;
+    public static final int INDEXEDCOLUMNTYPEORDINAL_FIELD_NUMBER = 5;
+    private java.util.List<java.lang.Integer> indexedColumnTypeOrdinal_;
+    /**
+     * <code>repeated int32 indexedColumnTypeOrdinal = 5;</code>
+     */
+    public java.util.List<java.lang.Integer>
+        getIndexedColumnTypeOrdinalList() {
+      return indexedColumnTypeOrdinal_;
+    }
+    /**
+     * <code>repeated int32 indexedColumnTypeOrdinal = 5;</code>
+     */
+    public int getIndexedColumnTypeOrdinalCount() {
+      return indexedColumnTypeOrdinal_.size();
+    }
+    /**
+     * <code>repeated int32 indexedColumnTypeOrdinal = 5;</code>
+     */
+    public int getIndexedColumnTypeOrdinal(int index) {
+      return indexedColumnTypeOrdinal_.get(index);
+    }
+
+    // repeated .ColumnReference dataTableColRefForCoveredColumns = 6;
+    public static final int DATATABLECOLREFFORCOVEREDCOLUMNS_FIELD_NUMBER = 6;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> dataTableColRefForCoveredColumns_;
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> getDataTableColRefForCoveredColumnsList() {
+      return dataTableColRefForCoveredColumns_;
+    }
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder> 
+        getDataTableColRefForCoveredColumnsOrBuilderList() {
+      return dataTableColRefForCoveredColumns_;
+    }
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    public int getDataTableColRefForCoveredColumnsCount() {
+      return dataTableColRefForCoveredColumns_.size();
+    }
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getDataTableColRefForCoveredColumns(int index) {
+      return dataTableColRefForCoveredColumns_.get(index);
+    }
+    /**
+     * <code>repeated .ColumnReference dataTableColRefForCoveredColumns = 6;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder getDataTableColRefForCoveredColumnsOrBuilder(
+        int index) {
+      return dataTableColRefForCoveredColumns_.get(index);
+    }
+
+    // repeated .ColumnReference indexTableColRefForCoveredColumns = 7;
+    public static final int INDEXTABLECOLREFFORCOVEREDCOLUMNS_FIELD_NUMBER = 7;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> indexTableColRefForCoveredColumns_;
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference> getIndexTableColRefForCoveredColumnsList() {
+      return indexTableColRefForCoveredColumns_;
+    }
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder> 
+        getIndexTableColRefForCoveredColumnsOrBuilderList() {
+      return indexTableColRefForCoveredColumns_;
+    }
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    public int getIndexTableColRefForCoveredColumnsCount() {
+      return indexTableColRefForCoveredColumns_.size();
+    }
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReference getIndexTableColRefForCoveredColumns(int index) {
+      return indexTableColRefForCoveredColumns_.get(index);
+    }
+    /**
+     * <code>repeated .ColumnReference indexTableColRefForCoveredColumns = 7;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnReferenceOrBuilder getIndexTableColRefForCoveredColumnsOrBuilder(
+        int index) {
+      return indexTableColRefForCoveredColumns_.get(index);
+    }
+
+    // required bool isLocalIndex = 8;
+    public static final int ISLOCALINDEX_FIELD_NUMBER = 8;
+    private boolean isLocalIndex_;
+    /**
+     * <code>required bool isLocalIndex = 8;</code>
+     */
+    public boolean hasIsLocalIndex() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>required bool isLocalIndex = 8;</code>
+     */
+    public boolean getIsLocalIndex() {
+      return isLocalIndex_;
+    }
+
+    // required bytes indexTableName = 9;
+    public static final int INDEXTABLENAME_FIELD_NUMBER = 9;
+    private com.google.protobuf.ByteString indexTableName_;
+    /**
+     * <code>required bytes indexTableName = 9;</code>
+     */
+    public boolean hasIndexTableName() {
+      return ((bitField0_ & 0x00000010) == 0x00000010);
+    }
+    /**
+     * <code>required bytes indexTableName = 9;</code>
+     */
+    public com.google.protobuf.ByteString getIndexTableName() {
+      return indexTableName_;
+    }
+
+    // required bool rowKeyOrderOptimizable = 10;
+    public static final int ROWKEYORDEROPTIMIZABLE_FIELD_NUMBER = 10;
+    private boolean rowKeyOrderOptimizable_;
+    /**
+     * <code>required bool rowKeyOrderOptimizable = 10;</code>
+     */
+    public boolean hasRowKeyOrderOptimizable() {
+      return ((bitField0_ & 0x00000020) == 0x00000020);
+    }
+    /**
+     * <code>required bool rowKeyOrderOptimizable = 10;</code>
+     */
+    public boolean getRowKeyOrderOptimizable() {
+      return rowKeyOrderOptimizable_;
+    }
+
+    // required bytes dataTableEmptyKeyValueColFamily = 11;
+    public static final int DATATABLEEMPTYKEYVALUECOLFAMILY_FIELD_NUMBER = 11;
+    private com.google.protobuf.ByteString dataTableEmptyKeyValueColFamily_;
+    /**
+     * <code>required bytes dataTableEmptyKeyValueColFamily = 11;</code>
+     */
+    public boolean hasDataTableEmptyKeyValueColFamily() {
+      return ((bitField0_ & 0x00000040) == 0x00000040);
+    }
+    /**
+     * <code>required bytes dataTableEmptyKeyValueColFamily = 11;</code>
+     */
+    public com.google.protobuf.ByteString getDataTableEmptyKeyValueColFamily() {
+      return dataTableEmptyKeyValueColFamily_;
+    }
+
+    // required .ImmutableBytesWritable emptyKeyValueColFamily = 12;
+    public static final int EMPTYKEYVALUECOLFAMILY_FIELD_NUMBER = 12;
+    private org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritable emptyKeyValueColFamily_;
+    /**
+     * <code>required .ImmutableBytesWritable emptyKeyValueColFamily = 12;</code>
+     */
+    public boolean hasEmptyKeyValueColFamily() {
+      return ((bitField0_ & 0x00000080) == 0x00000080);
+    }
+    /**
+     * <code>required .ImmutableBytesWritable emptyKeyValueColFamily = 12;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritable getEmptyKeyValueColFamily() {
+      return emptyKeyValueColFamily_;
+    }
+    /**
+     * <code>required .ImmutableBytesWritable emptyKeyValueColFamily = 12;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritableOrBuilder getEmptyKeyValueColFamilyOrBuilder() {
+      return emptyKeyValueColFamily_;
+    }
+
+    // optional bytes indexedExpressions = 13;
+    public static final int INDEXEDEXPRESSIONS_FIELD_NUMBER = 13;
+    private com.google.protobuf.ByteString indexedExpressions_;
+    /**
+     * <code>optional bytes indexedExpressions = 13;</code>
+     */
+    public boolean hasIndexedExpressions() {
+      return ((bitField0_ & 0x00000100) == 0x00000100);
+    }
+    /**
+     * <code>optional bytes indexedExpressions = 13;</code>
+     */
+    public com.google.protobuf.ByteString getIndexedExpressions() {
+      return indexedExpressions_;
+    }
+
+    // required bytes rowKeyMetadata = 14;
+    public static final int ROWKEYMETADATA_FIELD_NUMBER = 14;
+    private com.google.protobuf.ByteString rowKeyMetadata_;
+    /**
+     * <code>required bytes rowKeyMetadata = 14;</code>
+     */
+    public boolean hasRowKeyMetadata() {
+      return ((bitField0_ & 0x00000200) == 0x00000200);
+    }
+    /**
+     * <code>required bytes rowKeyMetadata = 14;</code>
+     */
+    public com.google.protobuf.ByteString getRowKeyMetadata() {
+      return rowKeyMetadata_;
+    }
+
+    // required int32 numDataTableColFamilies = 15;
+    public static final int NUMDATATABLECOLFAMILIES_FIELD_NUMBER = 15;
+    private int numDataTableColFamilies_;
+    /**
+     * <code>required int32 numDataTableColFamilies = 15;</code>
+     */
+    public boolean hasNumDataTableColFamilies() {
+      return ((bitField0_ & 0x00000400) == 0x00000400);
+    }
+    /**
+     * <code>required int32 numDataTableColFamilies = 15;</code>
+     */
+    public int getNumDataTableColFamilies() {
+      return numDataTableColFamilies_;
+    }
+
+    // required bool indexWalDisabled = 16;
+    public static final int INDEXWALDISABLED_FIELD_NUMBER = 16;
+    private boolean indexWalDisabled_;
+    /**
+     * <code>required bool indexWalDisabled = 16;</code>
+     */
+    public boolean hasIndexWalDisabled() {
+      return ((bitField0_ & 0x00000800) == 0x00000800);
+    }
+    /**
+     * <code>required bool indexWalDisabled = 16;</code>
+     */
+    public boolean getIndexWalDisabled() {
+      return indexWalDisabled_;
+    }
+
+    // required int32 indexRowKeyByteSize = 17;
+    public static final int INDEXROWKEYBYTESIZE_FIELD_NUMBER = 17;
+    private int indexRowKeyByteSize_;
+    /**
+     * <code>required int32 indexRowKeyByteSize = 17;</code>
+     */
+    public boolean hasIndexRowKeyByteSize() {
+      return ((bitField0_ & 0x00001000) == 0x00001000);
+    }
+    /**
+     * <code>required int32 indexRowKeyByteSize = 17;</code>
+     */
+    public int getIndexRowKeyByteSize() {
+      return indexRowKeyByteSize_;
+    }
+
+    // required bool immutable = 18;
+    public static final int IMMUTABLE_FIELD_NUMBER = 18;
+    private boolean immutable_;
+    /**
+     * <code>required bool immutable = 18;</code>
+     */
+    public boolean hasImmutable() {
+      return ((bitField0_ & 0x00002000) == 0x00002000);
+    }
+    /**
+     * <code>required bool immutable = 18;</code>
+     */
+    public boolean getImmutable() {
+      return immutable_;
+    }
+
+    // repeated .ColumnInfo indexedColumnInfo = 19;
+    public static final int INDEXEDCOLUMNINFO_FIELD_NUMBER = 19;
+    private java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo> indexedColumnInfo_;
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    public java.util.List<org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo> getIndexedColumnInfoList() {
+      return indexedColumnInfo_;
+    }
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    public java.util.List<? extends org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfoOrBuilder> 
+        getIndexedColumnInfoOrBuilderList() {
+      return indexedColumnInfo_;
+    }
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    public int getIndexedColumnInfoCount() {
+      return indexedColumnInfo_.size();
+    }
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfo getIndexedColumnInfo(int index) {
+      return indexedColumnInfo_.get(index);
+    }
+    /**
+     * <code>repeated .ColumnInfo indexedColumnInfo = 19;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ColumnInfoOrBuilder getIndexedColumnInfoOrBuilder(
+        int index) {
+      return indexedColumnInfo_.get(index);
+    }
+
+    // required int32 encodingScheme = 20;
+    public static final int ENCODINGSCHEME_FIELD_NUMBER = 20;
+    private int encodingScheme_;
+    /**
+     * <code>required int32 encodingScheme = 20;</code>
+     */
+    public boolean hasEncodingScheme() {
+      return ((bitField0_ & 0x00004000) == 0x00004000);
+    }
+    /**
+     * <code>required int32 encodingScheme = 20;</code>
+     */
+    public int getEncodingScheme() {
+      return encodingScheme_;
+    }
+
+    // required int32 immutableStorageScheme = 21;
+    public static final int IMMUTABLESTORAGESCHEME_FIELD_NUMBER = 21;
+    private int immutableStorageScheme_;
+    /**
+     * <code>required int32 immutableStorageScheme = 21;</code>
+     */
+    public boolean hasImmutableStorageScheme() {
+      return ((bitField0_ & 0x00008000) == 0x00008000);
+    }
+    /**
+     * <code>required int32 immutableStorageScheme = 21;</code>
+     */
+    public int getImmutableStorageScheme() {
+      return immutableStorageScheme_;
+    }
+
+    private void initFields() {
+      saltBuckets_ = 0;
+      isMultiTenant_ = false;
+      viewIndexId_ = com.google.protobuf.ByteString.EMPTY;
+      indexedColumns_ = java.util.Collections.emptyList();
+      indexedColumnTypeOrdinal_ = java.util.Collections.emptyList();
+      dataTableColRefForCoveredColumns_ = java.util.Collections.emptyList();
+      indexTableColRefForCoveredColumns_ = java.util.Collections.emptyList();
+      isLocalIndex_ = false;
+      indexTableName_ = com.google.protobuf.ByteString.EMPTY;
+      rowKeyOrderOptimizable_ = false;
+      dataTableEmptyKeyValueColFamily_ = com.google.protobuf.ByteString.EMPTY;
+      emptyKeyValueColFamily_ = org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ImmutableBytesWritable.getDefaultInstance();
+      indexedExpressions_ = com.google.protobuf.ByteString.EMPTY;
+      rowKeyMetadata_ = com.google.protobuf.ByteString.EMPTY;
+      numDataTableColFamilies_ = 0;
+      indexWalDisabled_ = false;
+      indexRowKeyByteSize_ = 0;
+      immutable_ = false;
+      indexedColumnInfo_ = java.util.Collections.emptyList();
+      encodingScheme_ = 0;
+      immutableStorageScheme_ = 0;
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      if (!hasSaltBuckets()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasIsMultiTenant()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasIsLocalIndex()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasIndexTableName()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRowKeyOrderOptimizable()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasDataTableEmptyKeyValueColFamily()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasEmptyKeyValueColFamily()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasRowKeyMetadata()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasNumDataTableColFamilies()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasIndexWalDisabled()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasIndexRowKeyByteSize()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasImmutable()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasEncodingScheme()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      if (!hasImmutableStorageScheme()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getIndexedColumnsCount(); i++) {
+        if (!getIndexedColumns(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getDataTableColRefForCoveredColumnsCount(); i++) {
+        if (!getDataTableColRefForCoveredColumns(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      for (int i = 0; i < getIndexTableColRefForCoveredColumnsCount(); i++) {
+        if (!getIndexTableColRefForCoveredColumns(i).isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
+      if (!getEmptyKeyValueColFamily().isInitialized()) {
+        memoizedIsInitialized = 0;
+        return false;
+      }
+      for (int i = 0; i < getIndexedColumnInfoCount(); i++) {
+        if (!getIndexedColumnInfo(i).isInitialized()) {
+          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.writeInt32(1, saltBuckets_);
+      }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeBool(2, isMultiTenant_);
+      }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeBytes(3, viewIndexId_);
+      }
+      for (int i = 0; i < indexedColumns_.size(); i++) {
+        output.writeMessage(4, indexedColumns_.get(i));
+      }
+      for (int i = 0; i < indexedColumnTypeOrdinal_.size(); i++) {
+        output.writeInt32(5, indexedColumnTypeOrdinal_.get(i));
+      }
+      for (int i = 0; i < dataTableColRefForCoveredColumns_.size(); i++) {
+        output.writeMessage(6, dataTableColRefForCoveredColumns_.get(i));
+      }
+      for (int i = 0; i < indexTableColRefForCoveredColumns_.size(); i++) {
+        output.writeMessage(7, indexTableColRefForCoveredColumns_.get(i));
+      }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(8, isLocalIndex_);
+      }
+      if (((bitField0_ & 0x00000010) == 0x00000010)) {
+        output.writeBytes(9, indexTableName_);
+      }
+      if (((bitField0_ & 0x00000020) == 0x00000020)) {
+        output.writeBool(10, rowKeyOrderOptimizable_);
+      }
+      if (((bitField0_ & 0x00000040) == 0x00000040)) {
+        output.writeBytes(11, dataTableEmptyKeyValueColFamily_);
+      }
+      if (((bitField0_ & 0x00000080) == 0x00000080)) {
+        output.writeMessage(12, emptyKeyValueColFamily_);
+      }
+      if (((bitField0_ & 0x00000100) == 0x00000100)) {
+        output.writeBytes(13, indexedExpressions_);
+      }
+      if (((bitField0_ & 0x00000200) == 0x00000200)) {
+        output.writeBytes(14, rowKeyMetadata_);
+      }
+      if (((bitField0_ & 0x00000400) == 0x00000400)) {
+        output.writeInt32(15, numDataTableColFamilies_);
+      }
+      if (((bitField0_ & 0x00000800) == 0x00000800)) {
+        output.writeBool(16, indexWalDisabled_);
+      }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        output.writeInt32(17, indexRowKeyByteSize_);
+      }
+      if (((bitField0_ & 0x00002000) == 0x00002000)) {
+        output.writeBool(18, immutable_);
+      }
+      for (int i = 0; i < indexedColumnInfo_.size(); i++) {
+        output.writeMessage(19, indexedColumnInfo_.get(i));
+      }
+      if (((bitField0_ & 0x00004000) == 0x00004000)) {
+        output.writeInt32(20, encodingScheme_);
+      }
+      if (((bitField0_ & 0x00008000) == 0x00008000)) {
+        output.writeInt32(21, immutableStorageScheme_);
+  

<TRUNCATED>

[12/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
index 1399f6c..f37d09b 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TransactionIT.java
@@ -26,337 +26,110 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
-import java.sql.Date;
 import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
-import java.util.List;
 import java.util.Properties;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
-import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.HBaseAdmin;
-import org.apache.hadoop.hbase.client.HTableInterface;
-import org.apache.hadoop.hbase.client.Put;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
 import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
-import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableKey;
-import org.apache.phoenix.schema.types.PInteger;
-import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.StringUtil;
 import org.apache.phoenix.util.TestUtil;
 import org.apache.tephra.TxConstants;
-import org.junit.Ignore;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
+public class TransactionIT  extends ParallelStatsDisabledIT {
 
-public class TransactionIT extends ParallelStatsDisabledIT {
-    
     @Test
-    public void testReadOwnWrites() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        String selectSql = "SELECT * FROM "+ fullTableName;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn, fullTableName);
-            conn.setAutoCommit(false);
-            ResultSet rs = conn.createStatement().executeQuery(selectSql);
-            assertFalse(rs.next());
-            
-            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = conn.prepareStatement(upsert);
-            // upsert two rows
-            TestUtil.setRowKeyColumns(stmt, 1);
-            stmt.execute();
-            TestUtil.setRowKeyColumns(stmt, 2);
-            stmt.execute();
-            
-            // verify rows can be read even though commit has not been called
-            rs = conn.createStatement().executeQuery(selectSql);
-            TestUtil.validateRowKeyColumns(rs, 1);
-            TestUtil.validateRowKeyColumns(rs, 2);
-            assertFalse(rs.next());
-            
-            conn.commit();
-            
-            // verify rows can be read after commit
-            rs = conn.createStatement().executeQuery(selectSql);
-            TestUtil.validateRowKeyColumns(rs, 1);
-            TestUtil.validateRowKeyColumns(rs, 2);
-            assertFalse(rs.next());
-        }
+    public void testReCreateTxnTableAfterDroppingExistingNonTxnTable() throws SQLException {
+        String tableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        Statement stmt = conn.createStatement();
+        stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
+        stmt.execute("DROP TABLE " + tableName);
+        stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) TRANSACTIONAL=true");
+        stmt.execute("CREATE INDEX " + tableName + "_IDX ON " + tableName + " (v1) INCLUDE(v2)");
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName)).isTransactional());
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,  tableName + "_IDX")).isTransactional());
     }
     
     @Test
-    public void testTxnClosedCorrecty() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        String selectSql = "SELECT * FROM "+fullTableName;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn, fullTableName);
+    public void testRowTimestampDisabled() throws SQLException {
+        String tableName = generateUniqueName();
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
             conn.setAutoCommit(false);
-            ResultSet rs = conn.createStatement().executeQuery(selectSql);
-            assertFalse(rs.next());
-            
-            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = conn.prepareStatement(upsert);
-            // upsert two rows
-            TestUtil.setRowKeyColumns(stmt, 1);
-            stmt.execute();
-            TestUtil.setRowKeyColumns(stmt, 2);
-            stmt.execute();
-            
-            // verify rows can be read even though commit has not been called
-            rs = conn.createStatement().executeQuery(selectSql);
-            TestUtil.validateRowKeyColumns(rs, 1);
-            TestUtil.validateRowKeyColumns(rs, 2);
-            // Long currentTx = rs.unwrap(PhoenixResultSet.class).getCurrentRow().getValue(0).getTimestamp();
-            assertFalse(rs.next());
-            
-            conn.close();
-            // start new connection
-            // conn.createStatement().executeQuery(selectSql);
-            // assertFalse("This transaction should not be on the invalid transactions",
-            // txManager.getCurrentState().getInvalid().contains(currentTx));
-        }
-    }
-    
-    @Test
-    public void testAutoCommitQuerySingleTable() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn, fullTableName);
-            conn.setAutoCommit(true);
-            // verify no rows returned
-            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName);
-            assertFalse(rs.next());
-        }
-    }
-    
-    @Test
-    public void testAutoCommitQueryMultiTables() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn, fullTableName);
-            conn.setAutoCommit(true);
-            // verify no rows returned
-            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName + " a JOIN " + fullTableName + " b ON (a.long_pk = b.int_pk)");
-            assertFalse(rs.next());
-        } 
-    }
-    
-    @Test
-    public void testColConflicts() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
-                Connection conn2 = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn1, fullTableName);
-            conn1.setAutoCommit(false);
-            conn2.setAutoCommit(false);
-            String selectSql = "SELECT * FROM "+fullTableName;
-            conn1.setAutoCommit(false);
-            ResultSet rs = conn1.createStatement().executeQuery(selectSql);
-            assertFalse(rs.next());
-            // upsert row using conn1
-            String upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = conn1.prepareStatement(upsertSql);
-            TestUtil.setRowKeyColumns(stmt, 1);
-            stmt.setInt(7, 10);
-            stmt.execute();
-            // upsert row using conn2
-            stmt = conn2.prepareStatement(upsertSql);
-            TestUtil.setRowKeyColumns(stmt, 1);
-            stmt.setInt(7, 11);
-            stmt.execute();
-            
-            conn1.commit();
-            //second commit should fail
+            Statement stmt = conn.createStatement();
             try {
-                conn2.commit();
+                stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP)) TRANSACTIONAL=true");
                 fail();
-            }   
-            catch (SQLException e) {
-                assertEquals(e.getErrorCode(), SQLExceptionCode.TRANSACTION_CONFLICT_EXCEPTION.getErrorCode());
             }
-        }
-    }
-    
-    private void testRowConflicts(String fullTableName) throws Exception {
-        try (Connection conn1 = DriverManager.getConnection(getUrl());
-                Connection conn2 = DriverManager.getConnection(getUrl())) {
-            conn1.setAutoCommit(false);
-            conn2.setAutoCommit(false);
-            String selectSql = "SELECT * FROM "+fullTableName;
-            conn1.setAutoCommit(false);
-            ResultSet rs = conn1.createStatement().executeQuery(selectSql);
-            boolean immutableRows = conn1.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, fullTableName)).isImmutableRows();
-            assertFalse(rs.next());
-            // upsert row using conn1
-            String upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = conn1.prepareStatement(upsertSql);
-            TestUtil.setRowKeyColumns(stmt, 1);
-            stmt.setInt(7, 10);
-            stmt.execute();
-            // upsert row using conn2
-            upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, b.int_col2) VALUES(?, ?, ?, ?, ?, ?, ?)";
-            stmt = conn2.prepareStatement(upsertSql);
-            TestUtil.setRowKeyColumns(stmt, 1);
-            stmt.setInt(7, 11);
-            stmt.execute();
-            
-            conn1.commit();
-            //second commit should fail
+            catch(SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP.getErrorCode(), e.getErrorCode());
+            }
+            stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP))");
             try {
-                conn2.commit();
-                if (!immutableRows) fail();
-            }   
-            catch (SQLException e) {
-                if (immutableRows) fail();
-                assertEquals(e.getErrorCode(), SQLExceptionCode.TRANSACTION_CONFLICT_EXCEPTION.getErrorCode());
+                stmt.execute("ALTER TABLE " + tableName + " SET TRANSACTIONAL=true");
+                fail();
+            }
+            catch(SQLException e) {
+                assertEquals(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP.getErrorCode(), e.getErrorCode());
             }
         }
     }
     
     @Test
-    public void testRowConflictDetected() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        Connection conn = DriverManager.getConnection(getUrl());
-        TestUtil.createTransactionalTable(conn, fullTableName);
-        testRowConflicts(fullTableName);
-    }
-    
-    @Test
-    public void testNoConflictDetectionForImmutableRows() throws Exception {
-        String transTableName = generateUniqueName();
-        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        Connection conn = DriverManager.getConnection(getUrl());
-        TestUtil.createTransactionalTable(conn, fullTableName);
-        conn.createStatement().execute("ALTER TABLE " + fullTableName + " SET IMMUTABLE_ROWS=true");
-        testRowConflicts(fullTableName);
-    }
-    
-    @Test
-    public void testNonTxToTxTable() throws Exception {
-        String nonTxTableName = generateUniqueName();
-        String indexName = generateUniqueName() + "_IDX";
+    public void testTransactionalTableMetadata() throws SQLException {
 
-        Connection conn = DriverManager.getConnection(getUrl());
-        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
-        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (1)");
-        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (2, 'a')");
-        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (3, 'b')");
-        conn.commit();
-        
-        conn.createStatement().execute("CREATE INDEX " + indexName + " ON " + nonTxTableName + "(v)");
-        // Reset empty column value to an empty value like it is pre-transactions
-        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
-        List<Put>puts = Lists.newArrayList(new Put(PInteger.INSTANCE.toBytes(1)), new Put(PInteger.INSTANCE.toBytes(2)), new Put(PInteger.INSTANCE.toBytes(3)));
-        for (Put put : puts) {
-            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
-        }
-        htable.put(puts);
-        
-        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + " SET TRANSACTIONAL=true");
-        
-        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
-        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
-        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(indexName));
-        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String transactTableName = generateUniqueName();
+            Statement stmt = conn.createStatement();
+            stmt.execute("CREATE TABLE " + transactTableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " +
+                "TRANSACTIONAL=true");
+            conn.commit();
 
-        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (4, 'c')");
-        ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ k FROM " + nonTxTableName + " WHERE v IS NULL");
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,  nonTxTableName)).isTransactional());
-        assertTrue(rs.next());
-        assertEquals(1,rs.getInt(1));
-        assertFalse(rs.next());
-        conn.commit();
-        
-        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (5, 'd')");
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, indexName)).isTransactional());
-        assertTrue(rs.next());
-        assertEquals(1,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(2,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(3,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(4,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(5,rs.getInt(1));
-        assertFalse(rs.next());
-        conn.rollback();
-        
-        rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
-        assertTrue(rs.next());
-        assertEquals(1,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(2,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(3,rs.getInt(1));
-        assertTrue(rs.next());
-        assertEquals(4,rs.getInt(1));
-        assertFalse(rs.next());
+            DatabaseMetaData dbmd = conn.getMetaData();
+            ResultSet rs = dbmd.getTables(null, null, StringUtil.escapeLike(transactTableName), null);
+            assertTrue(rs.next());
+            assertEquals("Transactional table was not marked as transactional in JDBC API.",
+                "true", rs.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
+
+            String nonTransactTableName = generateUniqueName();
+            Statement stmt2 = conn.createStatement();
+            stmt2.execute("CREATE TABLE " + nonTransactTableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) ");
+            conn.commit();
+
+            ResultSet rs2 = dbmd.getTables(null, null, StringUtil.escapeLike(nonTransactTableName), null);
+            assertTrue(rs2.next());
+            assertEquals("Non-transactional table was marked as transactional in JDBC API.",
+                "false", rs2.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
+        }
     }
     
-    @Ignore
     @Test
-    public void testNonTxToTxTableFailure() throws Exception {
-        String nonTxTableName = generateUniqueName();
-
-        Connection conn = DriverManager.getConnection(getUrl());
-        // Put table in SYSTEM schema to prevent attempts to update the cache after we disable SYSTEM.CATALOG
-        conn.createStatement().execute("CREATE TABLE SYSTEM." + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)");
-        conn.createStatement().execute("UPSERT INTO SYSTEM." + nonTxTableName + " VALUES (1)");
-        conn.commit();
-        // Reset empty column value to an empty value like it is pre-transactions
-        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
-        Put put = new Put(PInteger.INSTANCE.toBytes(1));
-        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
-        htable.put(put);
-        
-        HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
-        admin.disableTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
-        try {
-            // This will succeed initially in updating the HBase metadata, but then will fail when
-            // the SYSTEM.CATALOG table is attempted to be updated, exercising the code to restore
-            // the coprocessors back to the non transactional ones.
-            conn.createStatement().execute("ALTER TABLE SYSTEM." + nonTxTableName + " SET TRANSACTIONAL=true");
+    public void testOnDupKeyForTransactionalTable() throws Exception {
+        // TODO: we should support having a transactional table defined for a connectionless connection
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String transactTableName = generateUniqueName();
+            conn.createStatement().execute("CREATE TABLE " + transactTableName + " (k integer not null primary key, v bigint) TRANSACTIONAL=true");
+            conn.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES(0,0) ON DUPLICATE KEY UPDATE v = v + 1");
             fail();
         } catch (SQLException e) {
-            assertTrue(e.getMessage().contains(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " is disabled"));
-        } finally {
-            admin.enableTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
-            admin.close();
+            assertEquals(SQLExceptionCode.CANNOT_USE_ON_DUP_KEY_FOR_TRANSACTIONAL.getErrorCode(), e.getErrorCode());
         }
-        
-        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM SYSTEM." + nonTxTableName + " WHERE v IS NULL");
-        assertTrue(rs.next());
-        assertEquals(1,rs.getInt(1));
-        assertFalse(rs.next());
-        
-        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
-        assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
-        assertEquals(1,conn.unwrap(PhoenixConnection.class).getQueryServices().
-                getTableDescriptor(Bytes.toBytes("SYSTEM." + nonTxTableName)).
-                getFamily(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES).getMaxVersions());
     }
     
     @Test
@@ -439,120 +212,38 @@ public class TransactionIT extends ParallelStatsDisabledIT {
     }
     
     @Test
-    public void testCreateTableToBeTransactional() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        String t1 = generateUniqueName();
-        String t2 = generateUniqueName();
-        String ddl = "CREATE TABLE " + t1 + " (k varchar primary key) transactional=true";
-        conn.createStatement().execute(ddl);
-        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
-        PTable table = pconn.getTable(new PTableKey(null, t1));
-        HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
-        assertTrue(table.isTransactional());
-        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
-        
-        try {
-            ddl = "ALTER TABLE " + t1 + " SET transactional=false";
-            conn.createStatement().execute(ddl);
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX.getErrorCode(), e.getErrorCode());
-        }
-
-        HBaseAdmin admin = pconn.getQueryServices().getAdmin();
-        HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(t2));
-        desc.addFamily(new HColumnDescriptor(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES));
-        admin.createTable(desc);
-        ddl = "CREATE TABLE " + t2 + " (k varchar primary key) transactional=true";
-        conn.createStatement().execute(ddl);
-        assertEquals(Boolean.TRUE.toString(), admin.getTableDescriptor(TableName.valueOf(t2)).getValue(TxConstants.READ_NON_TX_DATA));
-        
-        // Should be ok, as HBase metadata should match existing metadata.
-        ddl = "CREATE TABLE IF NOT EXISTS " + t1 + " (k varchar primary key)"; 
-        try {
-            conn.createStatement().execute(ddl);
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX.getErrorCode(), e.getErrorCode());
-        }
-        ddl += " transactional=true";
-        conn.createStatement().execute(ddl);
-        table = pconn.getTable(new PTableKey(null, t1));
-        htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
-        assertTrue(table.isTransactional());
-        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
-    }
-
-    @Test
-    public void testCurrentDate() throws Exception {
+    public void testColConflicts() throws Exception {
         String transTableName = generateUniqueName();
         String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
-        String selectSql = "SELECT current_date() FROM "+fullTableName;
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            TestUtil.createTransactionalTable(conn, fullTableName);
-            conn.setAutoCommit(false);
-            ResultSet rs = conn.createStatement().executeQuery(selectSql);
+        try (Connection conn1 = DriverManager.getConnection(getUrl()); 
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            TestUtil.createTransactionalTable(conn1, fullTableName);
+            conn1.setAutoCommit(false);
+            conn2.setAutoCommit(false);
+            String selectSql = "SELECT * FROM "+fullTableName;
+            conn1.setAutoCommit(false);
+            ResultSet rs = conn1.createStatement().executeQuery(selectSql);
             assertFalse(rs.next());
-            
-            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
-            PreparedStatement stmt = conn.prepareStatement(upsert);
-            // upsert two rows
+            // upsert row using conn1
+            String upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
+            PreparedStatement stmt = conn1.prepareStatement(upsertSql);
             TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.setInt(7, 10);
+            stmt.execute();
+            // upsert row using conn2
+            stmt = conn2.prepareStatement(upsertSql);
+            TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.setInt(7, 11);
             stmt.execute();
-            conn.commit();
-            
-            rs = conn.createStatement().executeQuery(selectSql);
-            assertTrue(rs.next());
-            Date date1 = rs.getDate(1);
-            assertFalse(rs.next());
-            
-            Thread.sleep(1000);
             
-            rs = conn.createStatement().executeQuery(selectSql);
-            assertTrue(rs.next());
-            Date date2 = rs.getDate(1);
-            assertFalse(rs.next());
-            assertTrue("current_date() should change while executing multiple statements", date2.getTime() > date1.getTime());
-        }
-    }
-    
-    @Test
-    public void testReCreateTxnTableAfterDroppingExistingNonTxnTable() throws SQLException {
-        String tableName = generateUniqueName();
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.setAutoCommit(false);
-        Statement stmt = conn.createStatement();
-        stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)");
-        stmt.execute("DROP TABLE " + tableName);
-        stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) TRANSACTIONAL=true");
-        stmt.execute("CREATE INDEX " + tableName + "_IDX ON " + tableName + " (v1) INCLUDE(v2)");
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, tableName)).isTransactional());
-        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,  tableName + "_IDX")).isTransactional());
-    }
-    
-    @Test
-    public void testRowTimestampDisabled() throws SQLException {
-        String tableName = generateUniqueName();
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.setAutoCommit(false);
-            Statement stmt = conn.createStatement();
-            try {
-                stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP)) TRANSACTIONAL=true");
-                fail();
-            }
-            catch(SQLException e) {
-                assertEquals(SQLExceptionCode.CANNOT_CREATE_TXN_TABLE_WITH_ROW_TIMESTAMP.getErrorCode(), e.getErrorCode());
-            }
-            stmt.execute("CREATE TABLE " + tableName + "(k VARCHAR, v VARCHAR, d DATE NOT NULL, CONSTRAINT PK PRIMARY KEY(k,d ROW_TIMESTAMP))");
+            conn1.commit();
+            //second commit should fail
             try {
-                stmt.execute("ALTER TABLE " + tableName + " SET TRANSACTIONAL=true");
+                conn2.commit();
                 fail();
-            }
-            catch(SQLException e) {
-                assertEquals(SQLExceptionCode.CANNOT_ALTER_TO_BE_TXN_WITH_ROW_TIMESTAMP.getErrorCode(), e.getErrorCode());
+            }   
+            catch (SQLException e) {
+                assertEquals(e.getErrorCode(), SQLExceptionCode.TRANSACTION_CONFLICT_EXCEPTION.getErrorCode());
             }
         }
     }
@@ -600,118 +291,4 @@ public class TransactionIT extends ParallelStatsDisabledIT {
             conn.close();
         }
     }
-    
-    @Test
-    public void testParallelUpsertSelect() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
-        props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
-        props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        conn.setAutoCommit(false);
-        String fullTableName1 = generateUniqueName();
-        String fullTableName2 = generateUniqueName();
-        String sequenceName = "S_" + generateUniqueName();
-        conn.createStatement().execute("CREATE SEQUENCE " + sequenceName);
-        conn.createStatement().execute("CREATE TABLE " + fullTableName1 + " (pk INTEGER PRIMARY KEY, val INTEGER) SALT_BUCKETS=4,TRANSACTIONAL=true");
-        conn.createStatement().execute("CREATE TABLE " + fullTableName2 + " (pk INTEGER PRIMARY KEY, val INTEGER) TRANSACTIONAL=true");
-
-        for (int i = 0; i < 100; i++) {
-            conn.createStatement().execute("UPSERT INTO " + fullTableName1 + " VALUES (NEXT VALUE FOR " + sequenceName + ", " + (i%10) + ")");
-        }
-        conn.commit();
-        conn.setAutoCommit(true);
-        int upsertCount = conn.createStatement().executeUpdate("UPSERT INTO " + fullTableName2 + " SELECT pk, val FROM " + fullTableName1);
-        assertEquals(100,upsertCount);
-        conn.close();
-    }
-
-    @Test
-    public void testTransactionalTableMetadata() throws SQLException {
-
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String transactTableName = generateUniqueName();
-            Statement stmt = conn.createStatement();
-            stmt.execute("CREATE TABLE " + transactTableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " +
-                "TRANSACTIONAL=true");
-            conn.commit();
-
-            DatabaseMetaData dbmd = conn.getMetaData();
-            ResultSet rs = dbmd.getTables(null, null, StringUtil.escapeLike(transactTableName), null);
-            assertTrue(rs.next());
-            assertEquals("Transactional table was not marked as transactional in JDBC API.",
-                "true", rs.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
-
-            String nonTransactTableName = generateUniqueName();
-            Statement stmt2 = conn.createStatement();
-            stmt2.execute("CREATE TABLE " + nonTransactTableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) ");
-            conn.commit();
-
-            ResultSet rs2 = dbmd.getTables(null, null, StringUtil.escapeLike(nonTransactTableName), null);
-            assertTrue(rs2.next());
-            assertEquals("Non-transactional table was marked as transactional in JDBC API.",
-                "false", rs2.getString(PhoenixDatabaseMetaData.TRANSACTIONAL));
-        }
-    }
-
-    @Test
-    public void testInflightPartialEval() throws SQLException {
-
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String transactTableName = generateUniqueName();
-            Statement stmt = conn.createStatement();
-            stmt.execute("CREATE TABLE " + transactTableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " +
-                "TRANSACTIONAL=true");
-            
-            try (Connection conn1 = DriverManager.getConnection(getUrl()); Connection conn2 = DriverManager.getConnection(getUrl())) {
-                conn1.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES ('a','b','x')");
-                // Select to force uncommitted data to be written
-                ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + transactTableName);
-                assertTrue(rs.next());
-                assertEquals("a", rs.getString(1));
-                assertEquals("b", rs.getString(2));
-                assertFalse(rs.next());
-                
-                conn2.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES ('a','c','x')");
-                // Select to force uncommitted data to be written
-                rs = conn2.createStatement().executeQuery("SELECT * FROM " + transactTableName );
-                assertTrue(rs.next());
-                assertEquals("a", rs.getString(1));
-                assertEquals("c", rs.getString(2));
-                assertFalse(rs.next());
-                
-                // If the AndExpression were to see the uncommitted row from conn2, the filter would
-                // filter the row out early and no longer continue to evaluate other cells due to
-                // the way partial evaluation holds state.
-                rs = conn1.createStatement().executeQuery("SELECT * FROM " +  transactTableName + " WHERE v1 != 'c' AND v2 = 'x'");
-                assertTrue(rs.next());
-                assertEquals("a", rs.getString(1));
-                assertEquals("b", rs.getString(2));
-                assertFalse(rs.next());
-                
-                // Same as above for conn1 data
-                rs = conn2.createStatement().executeQuery("SELECT * FROM " + transactTableName + " WHERE v1 != 'b' AND v2 = 'x'");
-                assertTrue(rs.next());
-                assertEquals("a", rs.getString(1));
-                assertEquals("c", rs.getString(2));
-                assertFalse(rs.next());
-            }
-
-        }
-    }
-    
-    
-    @Test
-    public void testOnDupKeyForTransactionalTable() throws Exception {
-        // TODO: we should support having a transactional table defined for a connectionless connection
-        try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String transactTableName = generateUniqueName();
-            conn.createStatement().execute("CREATE TABLE " + transactTableName + " (k integer not null primary key, v bigint) TRANSACTIONAL=true");
-            conn.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES(0,0) ON DUPLICATE KEY UPDATE v = v + 1");
-            fail();
-        } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_USE_ON_DUP_KEY_FOR_TRANSACTIONAL.getErrorCode(), e.getErrorCode());
-        }
-    }
-    
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
index 246ecd4..cb3b4b3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/TxCheckpointIT.java
@@ -36,6 +36,7 @@ import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.tephra.Transaction.VisibilityLevel;
@@ -48,12 +49,25 @@ import org.junit.runners.Parameterized.Parameters;
 public class TxCheckpointIT extends ParallelStatsDisabledIT {
 	
 	private final boolean localIndex;
-	private final boolean mutable;
+	private final String tableDDLOptions;
 
-	public TxCheckpointIT(boolean localIndex, boolean mutable) {
+	public TxCheckpointIT(boolean localIndex, boolean mutable, boolean columnEncoded) {
+	    StringBuilder optionBuilder = new StringBuilder();
 		this.localIndex = localIndex;
-		this.mutable = mutable;
-
+		if (!columnEncoded) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
+        if (!mutable) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
+        this.tableDDLOptions = optionBuilder.toString();
 	}
 	
     private static Connection getConnection() throws SQLException {
@@ -66,10 +80,11 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         return conn;
     }
 	
-	@Parameters(name="TxCheckpointIT_localIndex={0},mutable={1}") // name is used by failsafe as file name in reports
+	@Parameters(name="TxCheckpointIT_localIndex={0},mutable={1},columnEncoded={2}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {     
-                 { false, false }, { false, true }, { true, false }, { true, true }  
+                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
+                 { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
            });
     }
     
@@ -86,7 +101,7 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         Connection conn = getConnection(props);
         conn.setAutoCommit(true);
         conn.createStatement().execute("CREATE SEQUENCE "+seqName);
-        conn.createStatement().execute("CREATE TABLE " + fullTableName + "(pk INTEGER PRIMARY KEY, val INTEGER)"+(!mutable? " IMMUTABLE_ROWS=true" : ""));
+        conn.createStatement().execute("CREATE TABLE " + fullTableName + "(pk INTEGER PRIMARY KEY, val INTEGER)" + tableDDLOptions);
         conn.createStatement().execute("CREATE "+(localIndex? "LOCAL " : "")+"INDEX " + indexName + " ON " + fullTableName + "(val)");
 
         conn.createStatement().execute("UPSERT INTO " + fullTableName + " VALUES (NEXT VALUE FOR " + seqName + ",1)");
@@ -117,12 +132,11 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
     }
     
     private void testRollbackOfUncommittedDelete(String indexDDL, String fullTableName) throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = getConnection();
         conn.setAutoCommit(false);
         try {
             Statement stmt = conn.createStatement();
-            stmt.execute("CREATE TABLE " + fullTableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)"+(!mutable? " IMMUTABLE_ROWS=true" : ""));
+            stmt.execute("CREATE TABLE " + fullTableName + "(k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
             stmt.execute(indexDDL);
             
             stmt.executeUpdate("upsert into " + fullTableName + " values('x1', 'y1', 'a1')");
@@ -206,13 +220,11 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
         String tableName = "TBL_" + generateUniqueName();
         String indexName = "IDX_" + generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(tableName, tableName);
-		Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
 		try (Connection conn = getConnection()) {
 			conn.setAutoCommit(false);
 			Statement stmt = conn.createStatement();
 
-			stmt.execute("CREATE TABLE " + fullTableName + "(ID BIGINT NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)"
-					+ (!mutable ? " IMMUTABLE_ROWS=true" : ""));
+			stmt.execute("CREATE TABLE " + fullTableName + "(ID BIGINT NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR)" + tableDDLOptions);
 			stmt.execute("CREATE " + (localIndex ? "LOCAL " : "")
 					+ "INDEX " + indexName + " ON " + fullTableName + " (v1) INCLUDE(v2)");
 
@@ -301,10 +313,8 @@ public class TxCheckpointIT extends ParallelStatsDisabledIT {
 		try (Connection conn = getConnection()) {
 			conn.setAutoCommit(false);
 			Statement stmt = conn.createStatement();
-			stmt.execute("CREATE TABLE " + fullTableName + "1(ID1 BIGINT NOT NULL PRIMARY KEY, FK1A INTEGER, FK1B INTEGER)"
-					+ (!mutable ? " IMMUTABLE_ROWS=true" : ""));
-			stmt.execute("CREATE TABLE " + fullTableName + "2(ID2 BIGINT NOT NULL PRIMARY KEY, FK2 INTEGER)"
-					+ (!mutable ? " IMMUTABLE_ROWS=true" : ""));
+			stmt.execute("CREATE TABLE " + fullTableName + "1(ID1 BIGINT NOT NULL PRIMARY KEY, FK1A INTEGER, FK1B INTEGER)" + tableDDLOptions);
+			stmt.execute("CREATE TABLE " + fullTableName + "2(ID2 BIGINT NOT NULL PRIMARY KEY, FK2 INTEGER)" + tableDDLOptions);
 			stmt.execute("CREATE " + (localIndex ? "LOCAL " : "")
 					+ "INDEX " + indexName + " ON " + fullTableName + "1 (FK1B)");
 			

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
index 0383251..18e4034 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/ServerCacheClient.java
@@ -56,6 +56,7 @@ import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.AddServerCac
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.RemoveServerCacheRequest;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.RemoveServerCacheResponse;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos.ServerCachingService;
+import org.apache.phoenix.index.IndexMaintainer;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.job.JobManager.JobCallable;
 import org.apache.phoenix.memory.MemoryManager.MemoryChunk;
@@ -215,6 +216,7 @@ public class ServerCacheClient {
                                                     }
                                                     builder.setCacheId(ByteStringer.wrap(cacheId));
                                                     builder.setCachePtr(org.apache.phoenix.protobuf.ProtobufUtil.toProto(cachePtr));
+                                                    builder.setHasProtoBufIndexMaintainer(true);
                                                     ServerCacheFactoryProtos.ServerCacheFactory.Builder svrCacheFactoryBuider = ServerCacheFactoryProtos.ServerCacheFactory.newBuilder();
                                                     svrCacheFactoryBuider.setClassName(cacheFactory.getClass().getName());
                                                     builder.setCacheFactory(svrCacheFactoryBuider.build());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
index 5c33967..d30f5dd 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCache.java
@@ -36,7 +36,7 @@ import org.apache.phoenix.memory.MemoryManager;
 public interface TenantCache {
     MemoryManager getMemoryManager();
     Closeable getServerCache(ImmutableBytesPtr cacheId);
-    Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory) throws SQLException;
+    Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory, boolean useProtoForIndexMaintainer) throws SQLException;
     void removeServerCache(ImmutableBytesPtr cacheId);
     void removeAllServerCache();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
index 658b4cc..3d178f6 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/cache/TenantCacheImpl.java
@@ -104,11 +104,11 @@ public class TenantCacheImpl implements TenantCache {
     }
     
     @Override
-    public Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory) throws SQLException {
+    public Closeable addServerCache(ImmutableBytesPtr cacheId, ImmutableBytesWritable cachePtr, byte[] txState, ServerCacheFactory cacheFactory, boolean useProtoForIndexMaintainer) throws SQLException {
         MemoryChunk chunk = this.getMemoryManager().allocate(cachePtr.getLength() + txState.length);
         boolean success = false;
         try {
-            Closeable element = cacheFactory.newCache(cachePtr, txState, chunk);
+            Closeable element = cacheFactory.newCache(cachePtr, txState, chunk, useProtoForIndexMaintainer);
             getServerCaches().put(cacheId, element);
             success = true;
             return element;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/CreateTableCompiler.java
index 07df105..b482998 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
@@ -38,6 +38,7 @@ import org.apache.phoenix.expression.IsNullExpression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
 import org.apache.phoenix.expression.visitor.StatelessTraverseNoExpressionVisitor;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
@@ -239,7 +240,7 @@ public class CreateTableCompiler {
         }
     }
     
-    private static class ViewWhereExpressionVisitor extends StatelessTraverseNoExpressionVisitor<Boolean> {
+    public static class ViewWhereExpressionVisitor extends StatelessTraverseNoExpressionVisitor<Boolean> {
         private boolean isUpdatable = true;
         private final PTable table;
         private int position;
@@ -318,13 +319,18 @@ public class CreateTableCompiler {
         @Override
         public Boolean visit(KeyValueColumnExpression node) {
             try {
-                this.position = table.getColumnFamily(node.getColumnFamily()).getColumn(node.getColumnName()).getPosition();
+                this.position = table.getColumnFamily(node.getColumnFamily()).getPColumnForColumnQualifier(node.getColumnQualifier()).getPosition();
             } catch (SQLException e) {
                 throw new RuntimeException(e); // Impossible
             }
             return Boolean.TRUE;
         }
         
+        @Override
+        public Boolean visit(SingleCellColumnExpression node) {
+            return visit(node.getKeyValueExpression());
+        }
+        
     }
     private static class VarbinaryDatum implements PDatum {
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
index 602cd6b..cee545a 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/DeleteCompiler.java
@@ -585,7 +585,7 @@ public class DeleteCompiler {
                             if (ptr.getLength() > 0) {
                                 byte[] uuidValue = ServerCacheClient.generateId();
                                 context.getScan().setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                                context.getScan().setAttribute(PhoenixIndexCodec.INDEX_MD, ptr.get());
+                                context.getScan().setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, ptr.get());
                                 context.getScan().setAttribute(BaseScannerRegionObserver.TX_STATE, txState);
                             }
                             ResultIterator iterator = aggPlan.iterator();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..fb4c542 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.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.TypeMismatchException;
-import org.apache.phoenix.schema.types.*;
+import org.apache.phoenix.schema.types.PArrayDataType;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PChar;
+import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.schema.types.PDate;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PDouble;
+import org.apache.phoenix.schema.types.PLong;
+import org.apache.phoenix.schema.types.PTimestamp;
+import org.apache.phoenix.schema.types.PUnsignedTimestamp;
+import org.apache.phoenix.schema.types.PVarbinary;
+import org.apache.phoenix.schema.types.PVarchar;
+import org.apache.phoenix.schema.types.PhoenixArray;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -386,7 +400,7 @@ public class ExpressionCompiler extends UnsupportedAllParseNodeVisitor<Expressio
     }
 
     protected void addColumn(PColumn column) {
-        context.getScan().addColumn(column.getFamilyName().getBytes(), column.getName().getBytes());
+        EncodedColumnsUtil.setColumns(column, context.getCurrentTable().getTable(), context.getScan());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..f401aad 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
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.util.ArrayList;
@@ -28,6 +30,7 @@ import java.util.Map;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.expression.Expression;
@@ -71,6 +74,8 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
@@ -125,10 +130,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;
         }
@@ -257,7 +264,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.getColumnQualifierBytes());
             projectedColumns.add(projectedColumn);
         }
         PTable t = PTableImpl.makePTable(table, projectedColumns);
@@ -332,26 +339,28 @@ 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) 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()));//, NON_ENCODED_QUALIFIERS));
+                }
             }
             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(),
+            // Storage scheme and encoding scheme don't matter here since the PTable is being used only for the purposes of create table.
+            // The actual values of these two will be determined by the metadata client.
+            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 +456,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().getColumnForColumnName(colName);
             return new ColumnRef(tableRef, column.getPosition());
 		}
 
@@ -672,7 +681,7 @@ public class FromCompiler {
                         familyName = PNameFactory.newName(family);
                     }
                     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, Bytes.toBytes(dynColumn.getColumnDefName().getColumnName())));
                     position++;
                 }
                 theTable = PTableImpl.makePTable(theTable, allcolumns);
@@ -774,16 +783,17 @@ public class FromCompiler {
                     // referenced by an outer wild-card select.
                     alias = String.valueOf(position);
                 }
+                PName name = PNameFactory.newName(alias);
                 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, name.getBytes());
                 columns.add(column);
             }
             PTable t = PTableImpl.makePTable(null, PName.EMPTY_NAME, PName.EMPTY_NAME, PTableType.SUBQUERY, null,
                     MetaDataProtocol.MIN_TABLE_TIMESTAMP, PTable.INITIAL_SEQ_NUM, null, null, columns, null, null,
                     Collections.<PTable> emptyList(), false, Collections.<PName> emptyList(), null, null, false, false,
                     false, null, null, null, false, false, 0, 0L, SchemaUtil
-                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false);
+                            .isNamespaceMappingEnabled(PTableType.SUBQUERY, connection.getQueryServices().getProps()), null, false, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
 
             String alias = subselectNode.getAlias();
             TableRef tableRef = new TableRef(alias, t, MetaDataProtocol.MIN_TABLE_TIMESTAMP, false);
@@ -858,7 +868,7 @@ public class FromCompiler {
                 while (iterator.hasNext()) {
                     TableRef tableRef = iterator.next();
                     try {
-                        PColumn column = tableRef.getTable().getColumn(colName);
+                        PColumn column = tableRef.getTable().getColumnForColumnName(colName);
                         if (theTableRef != null) { throw new AmbiguousColumnException(colName); }
                         theTableRef = tableRef;
                         theColumnPosition = column.getPosition();
@@ -871,12 +881,12 @@ public class FromCompiler {
             } else {
                 try {
                     TableRef tableRef = resolveTable(schemaName, tableName);
-                    PColumn column = tableRef.getTable().getColumn(colName);
+                    PColumn column = tableRef.getTable().getColumnForColumnName(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/d6a82e29/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..eef604b 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
@@ -17,6 +17,9 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
+import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
+
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -76,6 +79,8 @@ import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ProjectedColumn;
@@ -93,6 +98,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 +720,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);
                 }
             }
         }
@@ -1284,7 +1290,7 @@ public class JoinCompiler {
         if (type == JoinType.Full) {
             for (PColumn c : left.getColumns()) {
                 merged.add(new ProjectedColumn(c.getName(), c.getFamilyName(),
-                        c.getPosition(), true, ((ProjectedColumn) c).getSourceColumnRef()));
+                        c.getPosition(), true, ((ProjectedColumn) c).getSourceColumnRef(), SchemaUtil.isPKColumn(c) ? null : c.getName().getBytes()));
             }
         } else {
             merged.addAll(left.getColumns());
@@ -1294,14 +1300,13 @@ public class JoinCompiler {
             if (!SchemaUtil.isPKColumn(c)) {
                 PColumn column = new ProjectedColumn(c.getName(), c.getFamilyName(), 
                         position++, type == JoinType.Inner ? c.isNullable() : true, 
-                        ((ProjectedColumn) c).getSourceColumnRef());
+                        ((ProjectedColumn) c).getSourceColumnRef(), c.getName().getBytes());
                 merged.add(column);
             }
         }
         if (left.getBucketNum() != null) {
             merged.remove(0);
         }
-        
         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 +1315,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(), ONE_CELL_PER_COLUMN, NON_ENCODED_QUALIFIERS, PTable.EncodedCQCounter.NULL_COUNTER);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 b52e704..d975d35 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
@@ -57,6 +57,7 @@ import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
 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.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
@@ -77,10 +78,11 @@ public class ListJarsQueryPlan implements QueryPlan {
     
     static {
         List<ExpressionProjector> projectedColumns = new ArrayList<ExpressionProjector>();
+        PName colName = PNameFactory.newName("jar_location");
         PColumn column =
-                new PColumnImpl(PNameFactory.newName("jar_location"), null,
+                new PColumnImpl(colName, null,
                         PVarchar.INSTANCE, null, null, false, 0, SortOrder.getDefault(), 0, null,
-                        false, null, false, false);
+                        false, null, false, false, colName.getBytes());
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostDDLCompiler.java
index 393499a..0b3de6e 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().getColumnForColumnName(colName);
                                 return new ColumnRef(tableRef, column.getPosition());
                             }
                             
@@ -213,6 +214,7 @@ public class PostDDLCompiler {
                         ScanUtil.setTimeRange(scan, scan.getTimeRange().getMin(), ts);
                         if (emptyCF != null) {
                             scan.setAttribute(BaseScannerRegionObserver.EMPTY_CF, emptyCF);
+                            scan.setAttribute(BaseScannerRegionObserver.EMPTY_COLUMN_QUALIFIER, EncodedColumnsUtil.getEmptyKeyValueInfo(tableRef.getTable()).getFirst());
                         }
                         ServerCache cache = null;
                         try {
@@ -236,11 +238,12 @@ public class PostDDLCompiler {
                                     // data empty column family to stay the same, while the index empty column family
                                     // changes.
                                     PColumn column = deleteList.get(0);
+                                    byte[] cq = column.getColumnQualifierBytes();
                                     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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
index 81dbe0d..7e3c3b2 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/compile/PostLocalIndexDDLCompiler.java
@@ -31,6 +31,7 @@ import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.jdbc.PhoenixStatement.Operation;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.tuple.Tuple;
@@ -77,12 +78,16 @@ public class PostLocalIndexDDLCompiler {
             // rows per region as a result. The value of the attribute will be our persisted
             // index maintainers.
             // Define the LOCAL_INDEX_BUILD as a new static in BaseScannerRegionObserver
-            scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD, ByteUtil.copyKeyBytesIfNecessary(ptr));
+            scan.setAttribute(BaseScannerRegionObserver.LOCAL_INDEX_BUILD_PROTO, ByteUtil.copyKeyBytesIfNecessary(ptr));
             // By default, we'd use a FirstKeyOnly filter as nothing else needs to be projected for count(*).
             // However, in this case, we need to project all of the data columns that contribute to the index.
             IndexMaintainer indexMaintainer = index.getIndexMaintainer(dataTable, connection);
             for (ColumnReference columnRef : indexMaintainer.getAllColumns()) {
-                scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
+                if (index.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS) {
+                    scan.addFamily(columnRef.getFamily());
+                } else {
+                    scan.addColumn(columnRef.getFamily(), columnRef.getQualifier());
+                }
             }
 
             // Go through MutationPlan abstraction so that we can create local indexes

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..200b06c 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
@@ -24,11 +24,9 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
-import java.util.Set;
 
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
@@ -44,14 +42,11 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
-import org.apache.phoenix.expression.aggregator.ClientAggregators;
-import org.apache.phoenix.expression.aggregator.ServerAggregators;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
 import org.apache.phoenix.expression.function.ArrayIndexFunction;
-import org.apache.phoenix.expression.function.SingleAggregateFunction;
 import org.apache.phoenix.expression.visitor.ExpressionVisitor;
 import org.apache.phoenix.expression.visitor.ProjectedColumnExpressionVisitor;
 import org.apache.phoenix.expression.visitor.ReplaceArrayFunctionExpressionVisitor;
-import org.apache.phoenix.expression.visitor.SingleAggregateFunctionVisitor;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.parse.AliasedNode;
 import org.apache.phoenix.parse.BindParseNode;
@@ -78,6 +73,7 @@ import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PDatum;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
@@ -92,9 +88,7 @@ import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.SizedUtil;
 
-import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
 
 
 /**
@@ -217,7 +211,7 @@ public class ProjectionCompiler {
             PColumn indexColumn = null;
             ColumnRef ref = null;
             try {
-                indexColumn = index.getColumn(indexColName);
+                indexColumn = index.getColumnForColumnName(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
             } catch (ColumnNotFoundException e) {
                 if (index.getIndexType() == IndexType.LOCAL) {
@@ -289,7 +283,7 @@ public class ProjectionCompiler {
             ColumnRef ref = null;
             String indexColumnFamily = null;
             try {
-                indexColumn = index.getColumn(indexColName);
+                indexColumn = index.getColumnForColumnName(indexColName);
                 ref = new ColumnRef(tableRef, indexColumn.getPosition());
                 indexColumnFamily = indexColumn.getFamilyName() == null ? null : indexColumn.getFamilyName().getString();
             } catch (ColumnNotFoundException e) {
@@ -484,11 +478,13 @@ public class ProjectionCompiler {
                 }
             } else {
                 for (byte[] cq : entry.getValue()) {
-                    PColumn column = family.getColumn(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;
-                }
+                    //if (!Bytes.equals(cq, ByteUtil.EMPTY_BYTE_ARRAY) || cq.length > 0) {
+                        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;
+                    }
+                //}
             }
         }
         boolean isProjectEmptyKeyValue = false;
@@ -663,7 +659,14 @@ 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();
+                             PTable table = context.getCurrentTable().getTable();
+                             KeyValueColumnExpression keyValueColumnExpression;
+                             if (table.getImmutableStorageScheme() != ImmutableStorageScheme.ONE_CELL_PER_COLUMN) {
+                                 keyValueColumnExpression = new SingleCellColumnExpression(col, col.getName().getString(), table.getEncodingScheme());
+                             } else {
+                                 keyValueColumnExpression = new KeyValueColumnExpression(col);
+                             }
                              indexKVs.add(keyValueColumnExpression);
                              copyOfChildren.set(0, keyValueColumnExpression);
                              Integer count = arrayExpressionCounts.get(expression);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 5e0977b..8dacf11 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
@@ -53,6 +53,7 @@ import org.apache.phoenix.parse.TraceStatement;
 import org.apache.phoenix.query.KeyRange;
 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.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
@@ -74,10 +75,11 @@ public class TraceQueryPlan implements QueryPlan {
     private static final RowProjector TRACE_PROJECTOR;
     static {
         List<ExpressionProjector> projectedColumns = new ArrayList<ExpressionProjector>();
+        PName colName = PNameFactory.newName(MetricInfo.TRACE.columnName);
         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, colName.getBytes());
         List<PColumn> columns = new ArrayList<PColumn>();
         columns.add(column);
         Expression expression =


[15/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
index b9fa15b..7f64fee 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/GroupByIT.java
@@ -21,7 +21,6 @@ import static org.apache.phoenix.util.TestUtil.A_VALUE;
 import static org.apache.phoenix.util.TestUtil.B_VALUE;
 import static org.apache.phoenix.util.TestUtil.C_VALUE;
 import static org.apache.phoenix.util.TestUtil.E_VALUE;
-import static org.apache.phoenix.util.TestUtil.ROW3;
 import static org.apache.phoenix.util.TestUtil.ROW5;
 import static org.apache.phoenix.util.TestUtil.ROW6;
 import static org.apache.phoenix.util.TestUtil.ROW7;
@@ -35,9 +34,7 @@ import java.sql.DriverManager;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.Statement;
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 
@@ -50,15 +47,14 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
-import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 
 @RunWith(Parameterized.class)
 public class GroupByIT extends BaseQueryIT {
 
-    public GroupByIT(String indexDDL) {
-        super(indexDDL);
+    public GroupByIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Parameters(name="GroupByIT_{index}") // name is used by failsafe as file name in reports
@@ -74,71 +70,9 @@ public class GroupByIT extends BaseQueryIT {
     	BaseQueryIT.doSetup(props);
     }
     
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testGroupByCondition() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement statement = conn.prepareStatement("SELECT count(*) FROM aTable WHERE organization_id=? GROUP BY a_integer=6");
-        statement.setString(1, tenantId);
-        ResultSet rs = statement.executeQuery();
-        assertValueEqualsResultSet(rs, Arrays.<Object>asList(1L,8L));
-        try {
-            statement = conn.prepareStatement("SELECT count(*),a_integer=6 FROM aTable WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null) GROUP BY a_integer=6");
-            statement.setString(1, tenantId);
-            rs = statement.executeQuery();
-            List<List<Object>> expectedResults = Lists.newArrayList(
-                    Arrays.<Object>asList(1L,false),
-                    Arrays.<Object>asList(1L,true));
-            assertValuesEqualsResultSet(rs, expectedResults);
-        } finally {
-            conn.close();
-        }
-
-        
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
-        conn = DriverManager.getConnection(getUrl(), props);
-        try {
-            statement = conn.prepareStatement("UPSERT into aTable(organization_id,entity_id,a_integer) values(?,?,null)");
-            statement.setString(1, tenantId);
-            statement.setString(2, ROW3);
-            statement.executeUpdate();
-            conn.commit();
-        } finally {
-            conn.close();
-        }
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 60));
-        conn = DriverManager.getConnection(getUrl(), props);
-        statement = conn.prepareStatement("SELECT count(*) FROM aTable WHERE organization_id=? GROUP BY a_integer=6");
-        statement.setString(1, tenantId);
-        rs = statement.executeQuery();
-        assertValueEqualsResultSet(rs, Arrays.<Object>asList(1L,1L,7L));
-        statement = conn.prepareStatement("SELECT a_integer, entity_id FROM aTable WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null)");
-        statement.setString(1, tenantId);
-        rs = statement.executeQuery();
-        List<List<Object>> expectedResults = Lists.newArrayList(
-                Arrays.<Object>asList(null,ROW3),
-                Arrays.<Object>asList(5,ROW5),
-                Arrays.<Object>asList(6,ROW6));
-        assertValuesEqualsResultSet(rs, expectedResults);
-        try {
-            statement = conn.prepareStatement("SELECT count(*),a_integer=6 FROM aTable WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null) GROUP BY a_integer=6");
-            statement.setString(1, tenantId);
-            rs = statement.executeQuery();
-            expectedResults = Lists.newArrayList(
-                    Arrays.<Object>asList(1L,null),
-                    Arrays.<Object>asList(1L,false),
-                    Arrays.<Object>asList(1L,true));
-            assertValuesEqualsResultSet(rs, expectedResults);
-        } finally {
-            conn.close();
-        }
-    }
-    
     @Test
     public void testNoWhereScan() throws Exception {
-        String query = "SELECT y_integer FROM aTable";
+        String query = "SELECT y_integer FROM " + tableName;
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -165,7 +99,7 @@ public class GroupByIT extends BaseQueryIT {
     @Test
     public void testGroupedAggregation() throws Exception {
         // Tests that you don't get an ambiguous column exception when using the same alias as the column name
-        String query = "SELECT a_string as a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string";
+        String query = "SELECT a_string as a_string, count(1), 'foo' FROM " + tableName + " WHERE organization_id=? GROUP BY a_string";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -193,7 +127,7 @@ public class GroupByIT extends BaseQueryIT {
 
     @Test
     public void testDistinctGroupedAggregation() throws Exception {
-        String query = "SELECT DISTINCT a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string, b_string ORDER BY a_string, count(1)";
+        String query = "SELECT DISTINCT a_string, count(1), 'foo' FROM " + tableName + " WHERE organization_id=? GROUP BY a_string, b_string ORDER BY a_string, count(1)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -235,7 +169,7 @@ public class GroupByIT extends BaseQueryIT {
 
     @Test
     public void testDistinctLimitedGroupedAggregation() throws Exception {
-        String query = "SELECT /*+ NO_INDEX */ DISTINCT a_string, count(1), 'foo' FROM atable WHERE organization_id=? GROUP BY a_string, b_string ORDER BY count(1) desc,a_string LIMIT 2";
+        String query = "SELECT /*+ NO_INDEX */ DISTINCT a_string, count(1), 'foo' FROM " + tableName + " WHERE organization_id=? GROUP BY a_string, b_string ORDER BY count(1) desc,a_string LIMIT 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -273,7 +207,7 @@ public class GroupByIT extends BaseQueryIT {
 
     @Test
     public void testDistinctUngroupedAggregation() throws Exception {
-        String query = "SELECT DISTINCT count(1), 'foo' FROM atable WHERE organization_id=?";
+        String query = "SELECT DISTINCT count(1), 'foo' FROM " + tableName + " WHERE organization_id=?";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -292,7 +226,7 @@ public class GroupByIT extends BaseQueryIT {
 
     @Test
     public void testGroupedLimitedAggregation() throws Exception {
-        String query = "SELECT a_string, count(1) FROM atable WHERE organization_id=? GROUP BY a_string LIMIT 2";
+        String query = "SELECT a_string, count(1) FROM " + tableName + " WHERE organization_id=? GROUP BY a_string LIMIT 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -315,8 +249,8 @@ public class GroupByIT extends BaseQueryIT {
     @Test
     public void testPointInTimeGroupedAggregation() throws Exception {
         String updateStmt = 
-            "upsert into " +
-            "ATABLE VALUES ('" + tenantId + "','" + ROW5 + "','" + C_VALUE +"')";
+            "upsert into " + tableName + 
+            " VALUES ('" + tenantId + "','" + ROW5 + "','" + C_VALUE +"')";
         // Override value that was set at creation time
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -332,8 +266,8 @@ public class GroupByIT extends BaseQueryIT {
         upsertConn = DriverManager.getConnection(url, props);
         upsertConn.setAutoCommit(true); // Test auto commit
         updateStmt = 
-            "upsert into " +
-            "ATABLE VALUES (?, ?, ?)";
+            "upsert into " + tableName +
+            " VALUES (?, ?, ?)";
         // Insert all rows at ts
         PreparedStatement pstmt = upsertConn.prepareStatement(updateStmt);
         pstmt.setString(1, tenantId);
@@ -342,7 +276,7 @@ public class GroupByIT extends BaseQueryIT {
         pstmt.execute(); // should commit too
         upsertConn.close();
         
-        String query = "SELECT a_string, count(1) FROM atable WHERE organization_id='" + tenantId + "' GROUP BY a_string";
+        String query = "SELECT a_string, count(1) FROM " + tableName + " WHERE organization_id='" + tenantId + "' GROUP BY a_string";
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
         Connection conn = DriverManager.getConnection(getUrl(), props);
         Statement statement = conn.createStatement();
@@ -362,7 +296,7 @@ public class GroupByIT extends BaseQueryIT {
 
     @Test
     public void testUngroupedAggregation() throws Exception {
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
+        String query = "SELECT count(1) FROM " + tableName + " WHERE organization_id=? and a_string = ?";
         String url = getUrl();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5)); // Execute query at ts + 5
@@ -396,7 +330,7 @@ public class GroupByIT extends BaseQueryIT {
 
     @Test
     public void testUngroupedAggregationNoWhere() throws Exception {
-        String query = "SELECT count(*) FROM atable";
+        String query = "SELECT count(*) FROM " + tableName;
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
@@ -419,8 +353,8 @@ public class GroupByIT extends BaseQueryIT {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection upsertConn = DriverManager.getConnection(url, props);
         String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName + 
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_STRING) " +
@@ -450,7 +384,7 @@ public class GroupByIT extends BaseQueryIT {
         stmt.execute();
         upsertConn.close();
         
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
+        String query = "SELECT count(1) FROM " + tableName + " WHERE organization_id=? and a_string = ?";
         // Specify CurrentSCN on URL with extra stuff afterwards (which should be ignored)
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 2) + ";foo=bar"; // Run query at timestamp 2 
         Connection conn = DriverManager.getConnection(url, props);
@@ -467,8 +401,8 @@ public class GroupByIT extends BaseQueryIT {
     @Test
     public void testPointInTimeUngroupedLimitedAggregation() throws Exception {
         String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
+            "upsert into " + tableName +
+            " (" +
             "    ORGANIZATION_ID, " +
             "    ENTITY_ID, " +
             "    A_STRING) " +
@@ -502,7 +436,7 @@ public class GroupByIT extends BaseQueryIT {
         stmt.execute();
         upsertConn.close();
 
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ? LIMIT 3";
+        String query = "SELECT count(1) FROM " + tableName + " WHERE organization_id=? and a_string = ? LIMIT 3";
         // Specify CurrentSCN on URL with extra stuff afterwards (which should be ignored)
         url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 2) + ";foo=bar"; // Run query at timestamp 2 
         Connection conn = DriverManager.getConnection(url, props);
@@ -517,60 +451,6 @@ public class GroupByIT extends BaseQueryIT {
     }
 
     @Test
-    public void testPointInTimeDeleteUngroupedAggregation() throws Exception {
-        String updateStmt = 
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_STRING) " +
-            "VALUES (?, ?, ?)";
-        
-        // Override value that was set at creation time
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-
-        // Remove column value at ts + 1 (i.e. equivalent to setting the value to null)
-        Connection conn = DriverManager.getConnection(url, props);
-        PreparedStatement stmt = conn.prepareStatement(updateStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW7);
-        stmt.setString(3, null);
-        stmt.execute();
-        
-        // Delete row 
-        stmt = conn.prepareStatement("delete from atable where organization_id=? and entity_id=?");
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW5);
-        stmt.execute();
-        conn.commit();
-        conn.close();
-        
-        // Delete row at timestamp 3. This should not be seen by the query executing
-        // Remove column value at ts + 1 (i.e. equivalent to setting the value to null)
-        Connection futureConn = DriverManager.getConnection(getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3), props);
-        stmt = futureConn.prepareStatement("delete from atable where organization_id=? and entity_id=?");
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW6);
-        stmt.execute();
-        futureConn.commit();
-        futureConn.close();
-
-        String query = "SELECT count(1) FROM atable WHERE organization_id=? and a_string = ?";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
-        conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement statement = conn.prepareStatement(query);
-        statement.setString(1, tenantId);
-        statement.setString(2, B_VALUE);
-        ResultSet rs = statement.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(2, rs.getLong(1));
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-
-    @Test
     public void testGroupByWithIntegerDivision1() throws Exception {
         long ts = nextTimestamp();
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java
deleted file mode 100644
index 78be892..0000000
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropIT.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.end2end;
-
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Properties;
-
-import org.apache.phoenix.exception.SQLExceptionCode;
-import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableKey;
-import org.apache.phoenix.util.PropertiesUtil;
-import org.apache.phoenix.util.SchemaUtil;
-import org.junit.Test;
-
-public class ImmutableTablePropIT extends ParallelStatsDisabledIT {
-
-    @Test
-    public void testImmutableKeyword() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
-        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
-        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
-            Statement stmt = conn.createStatement();
-            // create table with immutable keyword
-            String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
-                    "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string)) STORE_NULLS=true";
-            stmt.execute(ddl);
-            
-            // create table without immutable keyword
-            ddl = "CREATE TABLE  " + mutableDataTableFullName +
-                    "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string)) STORE_NULLS=true";
-            stmt.execute(ddl);
-            
-            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
-            PTable immutableTable = phxConn.getTable(new PTableKey(null, immutableDataTableFullName));
-            assertTrue("IMMUTABLE_ROWS should be set to true", immutableTable.isImmutableRows());
-            PTable mutableTable = phxConn.getTable(new PTableKey(null, mutableDataTableFullName));
-            assertFalse("IMMUTABLE_ROWS should be set to false", mutableTable.isImmutableRows());
-        } 
-    }
-    
-    @Test
-    public void testImmutableProperty() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
-        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
-        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
-            Statement stmt = conn.createStatement();
-            // create table with immutable table property set to true
-            String ddl = "CREATE TABLE  " + immutableDataTableFullName +
-                    "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true";
-            stmt.execute(ddl);
-            
-            // create table with immutable table property set to false
-            ddl = "CREATE TABLE  " + mutableDataTableFullName +
-                    "  (a_string varchar not null, col1 integer" +
-                    "  CONSTRAINT pk PRIMARY KEY (a_string))  IMMUTABLE_ROWS=false";
-            stmt.execute(ddl);
-            
-            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
-            PTable immutableTable = phxConn.getTable(new PTableKey(null, immutableDataTableFullName));
-            assertTrue("IMMUTABLE_ROWS should be set to true", immutableTable.isImmutableRows());
-            PTable mutableTable = phxConn.getTable(new PTableKey(null, mutableDataTableFullName));
-            assertFalse("IMMUTABLE_ROWS should be set to false", mutableTable.isImmutableRows());
-        } 
-    }
-    
-    @Test
-    public void testImmutableKeywordAndProperty() throws Exception {
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
-        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
-        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
-            Statement stmt = conn.createStatement();
-            try {
-                // create immutable table with immutable table property set to true 
-                String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
-                        "  (a_string varchar not null, col1 integer" +
-                        "  CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true";
-                stmt.execute(ddl);
-                fail();
-            }
-            catch (SQLException e) {
-                assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode());
-            }
-            
-            try {
-                // create immutable table with immutable table property set to false
-                String ddl = "CREATE IMMUTABLE TABLE  " + mutableDataTableFullName +
-                        "  (a_string varchar not null, col1 integer" +
-                        "  CONSTRAINT pk PRIMARY KEY (a_string))  IMMUTABLE_ROWS=false";
-                stmt.execute(ddl);
-                fail();
-            }
-            catch (SQLException e) {
-                assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode());
-            }
-            
-        } 
-    }
-    
-}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java
new file mode 100644
index 0000000..52cfe9c
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ImmutableTablePropertiesIT.java
@@ -0,0 +1,189 @@
+/*
+ * 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Properties;
+
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.Test;
+
+public class ImmutableTablePropertiesIT extends ParallelStatsDisabledIT {
+
+    @Test
+    public void testImmutableKeyword() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create table with immutable keyword
+            String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) STORE_NULLS=true";
+            stmt.execute(ddl);
+            
+            // create table without immutable keyword
+            ddl = "CREATE TABLE  " + mutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) STORE_NULLS=true";
+            stmt.execute(ddl);
+            
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable immutableTable = phxConn.getTable(new PTableKey(null, immutableDataTableFullName));
+            assertTrue("IMMUTABLE_ROWS should be set to true", immutableTable.isImmutableRows());
+            PTable mutableTable = phxConn.getTable(new PTableKey(null, mutableDataTableFullName));
+            assertFalse("IMMUTABLE_ROWS should be set to false", mutableTable.isImmutableRows());
+        } 
+    }
+    
+    @Test
+    public void testImmutableProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create table with immutable table property set to true
+            String ddl = "CREATE TABLE  " + immutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true";
+            stmt.execute(ddl);
+            
+            // create table with immutable table property set to false
+            ddl = "CREATE TABLE  " + mutableDataTableFullName +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string))  IMMUTABLE_ROWS=false";
+            stmt.execute(ddl);
+            
+            PhoenixConnection phxConn = conn.unwrap(PhoenixConnection.class);
+            PTable immutableTable = phxConn.getTable(new PTableKey(null, immutableDataTableFullName));
+            assertTrue("IMMUTABLE_ROWS should be set to true", immutableTable.isImmutableRows());
+            PTable mutableTable = phxConn.getTable(new PTableKey(null, mutableDataTableFullName));
+            assertFalse("IMMUTABLE_ROWS should be set to false", mutableTable.isImmutableRows());
+        } 
+    }
+    
+    @Test
+    public void testImmutableKeywordAndProperty() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        String mutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            try {
+                // create immutable table with immutable table property set to true 
+                String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
+                        "  (a_string varchar not null, col1 integer" +
+                        "  CONSTRAINT pk PRIMARY KEY (a_string)) IMMUTABLE_ROWS=true";
+                stmt.execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode());
+            }
+            
+            try {
+                // create immutable table with immutable table property set to false
+                String ddl = "CREATE IMMUTABLE TABLE  " + mutableDataTableFullName +
+                        "  (a_string varchar not null, col1 integer" +
+                        "  CONSTRAINT pk PRIMARY KEY (a_string))  IMMUTABLE_ROWS=false";
+                stmt.execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.IMMUTABLE_TABLE_PROPERTY_INVALID.getErrorCode(), e.getErrorCode());
+            }
+            
+        } 
+    }
+    
+    @Test
+    public void testImmutableTableWithStorageSchemeAndColumnEncodingProps() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            try {
+                // create immutable table with immutable table property set to true 
+                String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName +
+                        "  (a_string varchar not null, col1 integer" +
+                        "  CONSTRAINT pk PRIMARY KEY (a_string)) COLUMN_ENCODED_BYTES=0, IMMUTABLE_STORAGE_SCHEME="
+                        + PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+                stmt.execute(ddl);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_AND_COLUMN_QUALIFIER_BYTES.getErrorCode(), e.getErrorCode());
+            }
+        } 
+    }
+    
+    @Test
+    public void testAlterImmutableStorageSchemeProp() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        String immutableDataTableFullName1 = SchemaUtil.getTableName("", generateUniqueName());
+        String immutableDataTableFullName2 = SchemaUtil.getTableName("", generateUniqueName());
+        try (Connection conn = DriverManager.getConnection(getUrl(), props);) {
+            Statement stmt = conn.createStatement();
+            // create an immutable table with  ONE_CELL_PER_COLUMN storage scheme
+            String ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName1 +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) COLUMN_ENCODED_BYTES=0, IMMUTABLE_STORAGE_SCHEME="
+                    + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN;
+            stmt.execute(ddl);
+            // create an immutable table with  SINGLE_CELL_ARRAY_WITH_OFFSETS storage scheme
+            ddl = "CREATE IMMUTABLE TABLE  " + immutableDataTableFullName2 +
+                    "  (a_string varchar not null, col1 integer" +
+                    "  CONSTRAINT pk PRIMARY KEY (a_string)) COLUMN_ENCODED_BYTES=4, IMMUTABLE_STORAGE_SCHEME="
+                    + PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
+            stmt.execute(ddl);
+            
+            // changing the storage scheme from/to ONCE_CELL_PER_COLUMN should fail
+            try {
+                stmt.execute("ALTER TABLE " + immutableDataTableFullName1 + " SET IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE.getErrorCode(), e.getErrorCode());
+            }
+            try {
+                stmt.execute("ALTER TABLE " + immutableDataTableFullName2 + " SET IMMUTABLE_STORAGE_SCHEME=" + PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+                fail();
+            }
+            catch (SQLException e) {
+                assertEquals(SQLExceptionCode.INVALID_IMMUTABLE_STORAGE_SCHEME_CHANGE.getErrorCode(), e.getErrorCode());
+            }
+        } 
+    }
+    
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java
new file mode 100644
index 0000000..da36288
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MutableQueryIT.java
@@ -0,0 +1,424 @@
+/*
+ * 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.end2end;
+
+import static org.apache.phoenix.util.TestUtil.A_VALUE;
+import static org.apache.phoenix.util.TestUtil.B_VALUE;
+import static org.apache.phoenix.util.TestUtil.C_VALUE;
+import static org.apache.phoenix.util.TestUtil.E_VALUE;
+import static org.apache.phoenix.util.TestUtil.ROW1;
+import static org.apache.phoenix.util.TestUtil.ROW3;
+import static org.apache.phoenix.util.TestUtil.ROW4;
+import static org.apache.phoenix.util.TestUtil.ROW5;
+import static org.apache.phoenix.util.TestUtil.ROW6;
+import static org.apache.phoenix.util.TestUtil.ROW7;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+public class MutableQueryIT extends BaseQueryIT {
+    
+    @Parameters(name="indexDDL={0},mutable={1},columnEncoded={2}")
+    @Shadower(classBeingShadowed = BaseQueryIT.class)
+    public static Collection<Object> data() {
+        List<Object> testCases = Lists.newArrayList();
+        for (String indexDDL : INDEX_DDLS) {
+            for (boolean columnEncoded : new boolean[]{false,true}) {
+                testCases.add(new Object[] { indexDDL, true, columnEncoded });
+            }
+        }
+        return testCases;
+    }
+    
+    @BeforeClass
+    @Shadower(classBeingShadowed = BaseQueryIT.class)
+    public static void doSetup() throws Exception {
+        Map<String,String> props = Maps.newHashMapWithExpectedSize(3);
+        props.put(QueryServices.DEFAULT_KEEP_DELETED_CELLS_ATTRIB, Boolean.TRUE.toString());
+        BaseQueryIT.doSetup(props);
+    }
+
+    public MutableQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
+    }
+    
+    @Test
+    public void testSumOverNullIntegerColumn() throws Exception {
+        String query = "SELECT sum(a_integer) FROM " + tableName + " a";
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (organization_id,entity_id,a_integer) VALUES('" + getOrganizationId() + "','" + ROW3 + "',NULL)");
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, tableName);
+        conn1.close();
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 50));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(42, rs.getInt(1));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 70));
+        conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(true);
+        conn.createStatement().execute("UPSERT INTO " + tableName + " (organization_id,entity_id,a_integer) SELECT organization_id, entity_id, CAST(null AS integer) FROM " + tableName);
+
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 90));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(0, rs.getInt(1));
+            assertTrue(rs.wasNull());
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+    
+    private void testNoStringValue(String value) throws Exception {
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(
+                "upsert into " + tableName + " VALUES (?, ?, ?)"); // without specifying columns
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW5);
+        stmt.setString(3, value);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn1 = DriverManager.getConnection(getUrl(), props);
+        analyzeTable(conn1, tableName);
+        conn1.close();
+        
+        String query = "SELECT a_string, b_string FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 30));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            PreparedStatement statement = conn.prepareStatement(query);
+            statement.setString(1, tenantId);
+            ResultSet rs = statement.executeQuery();
+            assertTrue (rs.next());
+            assertEquals(null, rs.getString(1));
+            assertTrue(rs.wasNull());
+            assertEquals(C_VALUE, rs.getString("B_string"));
+            assertFalse(rs.next());
+        } finally {
+            conn.close();
+        }
+    }
+
+    @Test
+    public void testNullStringValue() throws Exception {
+        testNoStringValue(null);
+    }
+    
+    @Test
+    public void testEmptyStringValue() throws Exception {
+        testNoStringValue("");
+    }
+    
+    @Test
+    public void testUnfoundSingleColumnCaseStatement() throws Exception {
+        String query = "SELECT entity_id, b_string FROM " + tableName + " WHERE organization_id=? and CASE WHEN a_integer = 0 or a_integer != 0 THEN 1 ELSE 0 END = 0";
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 5); // Run query at timestamp 5
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(url, props);
+        // Set ROW5.A_INTEGER to null so that we have one row
+        // where the else clause of the CASE statement will
+        // fire.
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String upsertStmt =
+            "upsert into " + tableName +
+            " (" +
+            "    ENTITY_ID, " +
+            "    ORGANIZATION_ID, " +
+            "    A_INTEGER) " +
+            "VALUES ('" + ROW5 + "','" + tenantId + "', null)";
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(ROW5, rs.getString(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testGroupByCondition() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement("SELECT count(*) FROM " + tableName + " WHERE organization_id=? GROUP BY a_integer=6");
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        assertValueEqualsResultSet(rs, Arrays.<Object>asList(1L,8L));
+        try {
+            statement = conn.prepareStatement("SELECT count(*),a_integer=6 FROM " + tableName + " WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null) GROUP BY a_integer=6");
+            statement.setString(1, tenantId);
+            rs = statement.executeQuery();
+            List<List<Object>> expectedResults = Lists.newArrayList(
+                    Arrays.<Object>asList(1L,false),
+                    Arrays.<Object>asList(1L,true));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+
+        
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 40));
+        conn = DriverManager.getConnection(getUrl(), props);
+        try {
+            statement = conn.prepareStatement("UPSERT into " + tableName + " (organization_id,entity_id,a_integer) values(?,?,null)");
+            statement.setString(1, tenantId);
+            statement.setString(2, ROW3);
+            statement.executeUpdate();
+            conn.commit();
+        } finally {
+            conn.close();
+        }
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 60));
+        conn = DriverManager.getConnection(getUrl(), props);
+        statement = conn.prepareStatement("SELECT count(*) FROM " + tableName + " WHERE organization_id=? GROUP BY a_integer=6");
+        statement.setString(1, tenantId);
+        rs = statement.executeQuery();
+        assertValueEqualsResultSet(rs, Arrays.<Object>asList(1L,1L,7L));
+        statement = conn.prepareStatement("SELECT a_integer, entity_id FROM " + tableName + " WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null)");
+        statement.setString(1, tenantId);
+        rs = statement.executeQuery();
+        List<List<Object>> expectedResults = Lists.newArrayList(
+                Arrays.<Object>asList(null,ROW3),
+                Arrays.<Object>asList(5,ROW5),
+                Arrays.<Object>asList(6,ROW6));
+        assertValuesEqualsResultSet(rs, expectedResults);
+        try {
+            statement = conn.prepareStatement("SELECT count(*),a_integer=6 FROM " + tableName + " WHERE organization_id=? and (a_integer IN (5,6) or a_integer is null) GROUP BY a_integer=6");
+            statement.setString(1, tenantId);
+            rs = statement.executeQuery();
+            expectedResults = Lists.newArrayList(
+                    Arrays.<Object>asList(1L,null),
+                    Arrays.<Object>asList(1L,false),
+                    Arrays.<Object>asList(1L,true));
+            assertValuesEqualsResultSet(rs, expectedResults);
+        } finally {
+            conn.close();
+        }
+    }
+    
+    @Test
+    public void testPointInTimeDeleteUngroupedAggregation() throws Exception {
+        String updateStmt = 
+            "upsert into " + tableName +
+            " (" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_STRING) " +
+            "VALUES (?, ?, ?)";
+        
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+
+        // Remove column value at ts + 1 (i.e. equivalent to setting the value to null)
+        Connection conn = DriverManager.getConnection(url, props);
+        PreparedStatement stmt = conn.prepareStatement(updateStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW7);
+        stmt.setString(3, null);
+        stmt.execute();
+        
+        // Delete row 
+        stmt = conn.prepareStatement("delete from " + tableName + " where organization_id=? and entity_id=?");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW5);
+        stmt.execute();
+        conn.commit();
+        conn.close();
+        
+        // Delete row at timestamp 3. This should not be seen by the query executing
+        // Remove column value at ts + 1 (i.e. equivalent to setting the value to null)
+        Connection futureConn = DriverManager.getConnection(getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3), props);
+        stmt = futureConn.prepareStatement("delete from " + tableName + " where organization_id=? and entity_id=?");
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW6);
+        stmt.execute();
+        futureConn.commit();
+        futureConn.close();
+
+        String query = "SELECT count(1) FROM " + tableName + " WHERE organization_id=? and a_string = ?";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
+        conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        statement.setString(2, B_VALUE);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(2, rs.getLong(1));
+        assertFalse(rs.next());
+        conn.close();
+    }
+    
+    @Test
+    public void testPointInTimeScan() throws Exception {
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String upsertStmt =
+            "upsert into " + tableName +
+            " (" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_INTEGER) " +
+            "VALUES (?, ?, ?)";
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW4);
+        stmt.setInt(3, 5);
+        stmt.execute(); // should commit too
+        
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);
+        Connection conn1 = DriverManager.getConnection(url, props);
+        analyzeTable(conn1, tableName);
+        conn1.close();
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 30);
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW4);
+        stmt.setInt(3, 9);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        String query = "SELECT organization_id, a_string AS a FROM " + tableName + " WHERE organization_id=? and a_integer = 5";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        assertTrue(rs.next());
+        assertEquals(tenantId, rs.getString(1));
+        assertEquals(A_VALUE, rs.getString("a"));
+        assertTrue(rs.next());
+        assertEquals(tenantId, rs.getString(1));
+        assertEquals(B_VALUE, rs.getString(2));
+        assertFalse(rs.next());
+        conn.close();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testPointInTimeLimitedScan() throws Exception {
+        // Override value that was set at creation time
+        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 1); // Run query at timestamp 5
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection upsertConn = DriverManager.getConnection(url, props);
+        String upsertStmt =
+            "upsert into " + tableName +
+            " (" +
+            "    ORGANIZATION_ID, " +
+            "    ENTITY_ID, " +
+            "    A_INTEGER) " +
+            "VALUES (?, ?, ?)";
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW1);
+        stmt.setInt(3, 6);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+
+        // Override value again, but should be ignored since it's past the SCN
+        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 3);
+        upsertConn = DriverManager.getConnection(url, props);
+        upsertConn.setAutoCommit(true); // Test auto commit
+        // Insert all rows at ts
+        stmt = upsertConn.prepareStatement(upsertStmt);
+        stmt.setString(1, tenantId);
+        stmt.setString(2, ROW1);
+        stmt.setInt(3, 0);
+        stmt.execute(); // should commit too
+        upsertConn.close();
+        
+        String query = "SELECT a_integer,b_string FROM " + tableName + " WHERE organization_id=? and a_integer <= 5 limit 2";
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        PreparedStatement statement = conn.prepareStatement(query);
+        statement.setString(1, tenantId);
+        ResultSet rs = statement.executeQuery();
+        List<List<Object>> expectedResultsA = Lists.newArrayList(
+                Arrays.<Object>asList(2, C_VALUE),
+                Arrays.<Object>asList( 3, E_VALUE));
+        List<List<Object>> expectedResultsB = Lists.newArrayList(
+                Arrays.<Object>asList( 5, C_VALUE),
+                Arrays.<Object>asList(4, B_VALUE));
+        // Since we're not ordering and we may be using a descending index, we don't
+        // know which rows we'll get back.
+        assertOneOfValuesEqualsResultSet(rs, expectedResultsA,expectedResultsB);
+       conn.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
index 3d0fa2c..965071f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NativeHBaseTypesIT.java
@@ -151,7 +151,7 @@ public class NativeHBaseTypesIT extends BaseClientManagedTimeIT {
         }
         // Create Phoenix table after HBase table was created through the native APIs
         // The timestamp of the table creation must be later than the timestamp of the data
-        ensureTableCreated(getUrl(),HBASE_NATIVE,HBASE_NATIVE,null, ts+1);
+        ensureTableCreated(getUrl(),HBASE_NATIVE,HBASE_NATIVE,null, ts+1, null);
     }
     
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
index 76627be..036d278 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/NotQueryIT.java
@@ -52,8 +52,8 @@ import com.google.common.primitives.Floats;
 @RunWith(Parameterized.class)
 public class NotQueryIT extends BaseQueryIT {
 
-    public NotQueryIT(String indexDDL) {
-        super(indexDDL);
+    public NotQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
     
     @Parameters(name="NotQueryIT_{index}") // name is used by failsafe as file name in reports
@@ -63,7 +63,7 @@ public class NotQueryIT extends BaseQueryIT {
     
     @Test
     public void testNotInList() throws Exception {
-        String query = "SELECT entity_id FROM aTable WHERE organization_id=? and entity_id NOT IN (?,?,?,?,?,?)";
+        String query = "SELECT entity_id FROM " + tableName + " WHERE organization_id=? and entity_id NOT IN (?,?,?,?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -91,7 +91,7 @@ public class NotQueryIT extends BaseQueryIT {
     
     @Test
     public void testNotInListOfFloat() throws Exception {
-        String query = "SELECT a_float FROM aTable WHERE organization_id=? and a_float NOT IN (?,?,?,?,?,?)";
+        String query = "SELECT a_float FROM " + tableName + " WHERE organization_id=? and a_float NOT IN (?,?,?,?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -119,7 +119,7 @@ public class NotQueryIT extends BaseQueryIT {
     
     @Test
     public void testNotInListOfDouble() throws Exception {
-        String query = "SELECT a_double FROM aTable WHERE organization_id=? and a_double NOT IN (?,?,?,?,?,?)";
+        String query = "SELECT a_double FROM " + tableName + " WHERE organization_id=? and a_double NOT IN (?,?,?,?,?,?)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -148,7 +148,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEquals() throws Exception {
         String query = "SELECT entity_id -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_integer != 1 and a_integer <= 2";
+        "FROM " + tableName + " WHERE organization_id=? and a_integer != 1 and a_integer <= 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -167,7 +167,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByTinyInt() throws Exception {
         String query = "SELECT a_byte -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_byte != 1 and a_byte <= 2";
+        "FROM " + tableName + " WHERE organization_id=? and a_byte != 1 and a_byte <= 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -186,7 +186,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsBySmallInt() throws Exception {
         String query = "SELECT a_short -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_short != 128 and a_short !=0 and a_short <= 129";
+        "FROM " + tableName + " WHERE organization_id=? and a_short != 128 and a_short !=0 and a_short <= 129";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -205,7 +205,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByFloat() throws Exception {
         String query = "SELECT a_float -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_float != CAST(0.01 AS FLOAT) and a_float <= CAST(0.02 AS FLOAT)";
+        "FROM " + tableName + " WHERE organization_id=? and a_float != CAST(0.01 AS FLOAT) and a_float <= CAST(0.02 AS FLOAT)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -224,7 +224,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByUnsignedFloat() throws Exception {
         String query = "SELECT a_unsigned_float -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_unsigned_float != 0.01 and a_unsigned_float <= 0.02";
+        "FROM " + tableName + " WHERE organization_id=? and a_unsigned_float != 0.01 and a_unsigned_float <= 0.02";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -243,7 +243,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByDouble() throws Exception {
         String query = "SELECT a_double -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_double != CAST(0.0001 AS DOUBLE) and a_double <= CAST(0.0002 AS DOUBLE)";
+        "FROM " + tableName + " WHERE organization_id=? and a_double != CAST(0.0001 AS DOUBLE) and a_double <= CAST(0.0002 AS DOUBLE)";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -262,7 +262,7 @@ public class NotQueryIT extends BaseQueryIT {
     @Test
     public void testNotEqualsByUnsignedDouble() throws Exception {
         String query = "SELECT a_unsigned_double -- and here comment\n" + 
-        "FROM aTable WHERE organization_id=? and a_unsigned_double != 0.0001 and a_unsigned_double <= 0.0002";
+        "FROM " + tableName + " WHERE organization_id=? and a_unsigned_double != 0.0001 and a_unsigned_double <= 0.0002";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -280,8 +280,8 @@ public class NotQueryIT extends BaseQueryIT {
 
     @Test
     public void testNotEquals2() throws Exception {
-        String query = "SELECT entity_id FROM // one more comment  \n" +
-        "aTable WHERE organization_id=? and not a_integer = 1 and a_integer <= 2";
+        String query = "SELECT entity_id FROM // one more comment  \n" + tableName + 
+        " WHERE organization_id=? and not a_integer = 1 and a_integer <= 2";
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 2)); // Execute at timestamp 2
         Connection conn = DriverManager.getConnection(getUrl(), props);

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
index af5a905..408bf18 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PercentileIT.java
@@ -511,9 +511,9 @@ public class PercentileIT extends ParallelStatsDisabledIT {
             Date date, Long ts) throws Exception {
         String tableName = generateUniqueName();
         if (ts == null) {
-            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits);
+            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, null);
         } else {
-            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, ts - 2);
+            ensureTableCreated(getUrl(), tableName, ATABLE_NAME, splits, ts - 2, null);
         }
 
         Properties props = new Properties();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
index 73eb2a3..bb1eabe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PointInTimeQueryIT.java
@@ -17,18 +17,13 @@
  */
 package org.apache.phoenix.end2end;
 
-import static org.apache.phoenix.util.TestUtil.A_VALUE;
-import static org.apache.phoenix.util.TestUtil.B_VALUE;
-import static org.apache.phoenix.util.TestUtil.ROW4;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
-import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.util.Map;
 import java.util.Properties;
@@ -44,8 +39,8 @@ import com.google.common.collect.Maps;
 
 public class PointInTimeQueryIT extends BaseQueryIT {
 
-    public PointInTimeQueryIT(String indexDDL) {
-        super(indexDDL);
+    public PointInTimeQueryIT(String indexDDL, boolean mutable, boolean columnEncoded) {
+        super(indexDDL, mutable, columnEncoded);
     }
 
     @BeforeClass
@@ -57,61 +52,6 @@ public class PointInTimeQueryIT extends BaseQueryIT {
     }
     
     @Test
-    public void testPointInTimeScan() throws Exception {
-        // Override value that was set at creation time
-        String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 10);
-        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-        Connection upsertConn = DriverManager.getConnection(url, props);
-        String upsertStmt =
-            "upsert into " +
-            "ATABLE(" +
-            "    ORGANIZATION_ID, " +
-            "    ENTITY_ID, " +
-            "    A_INTEGER) " +
-            "VALUES (?, ?, ?)";
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        PreparedStatement stmt = upsertConn.prepareStatement(upsertStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW4);
-        stmt.setInt(3, 5);
-        stmt.execute(); // should commit too
-        
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 15);
-        Connection conn1 = DriverManager.getConnection(url, props);
-        analyzeTable(conn1, "ATABLE");
-        conn1.close();
-        upsertConn.close();
-
-        // Override value again, but should be ignored since it's past the SCN
-        url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + (ts + 30);
-        upsertConn = DriverManager.getConnection(url, props);
-        upsertConn.setAutoCommit(true); // Test auto commit
-        // Insert all rows at ts
-        stmt = upsertConn.prepareStatement(upsertStmt);
-        stmt.setString(1, tenantId);
-        stmt.setString(2, ROW4);
-        stmt.setInt(3, 9);
-        stmt.execute(); // should commit too
-        upsertConn.close();
-        
-        String query = "SELECT organization_id, a_string AS a FROM atable WHERE organization_id=? and a_integer = 5";
-        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 20));
-        Connection conn = DriverManager.getConnection(getUrl(), props);
-        PreparedStatement statement = conn.prepareStatement(query);
-        statement.setString(1, tenantId);
-        ResultSet rs = statement.executeQuery();
-        assertTrue(rs.next());
-        assertEquals(tenantId, rs.getString(1));
-        assertEquals(A_VALUE, rs.getString("a"));
-        assertTrue(rs.next());
-        assertEquals(tenantId, rs.getString(1));
-        assertEquals(B_VALUE, rs.getString(2));
-        assertFalse(rs.next());
-        conn.close();
-    }
-
-    @Test
     public void testPointInTimeSequence() throws Exception {
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn;
@@ -122,7 +62,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         conn.createStatement().execute("CREATE SEQUENCE s");
         
         try {
-            conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+            conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
             fail();
         } catch (SequenceNotFoundException e) {
             conn.close();
@@ -130,14 +70,14 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+10));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         conn.close();
         
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+7));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(2, rs.getInt(1));
         conn.close();
@@ -145,7 +85,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+15));
         conn = DriverManager.getConnection(getUrl(), props);
         conn.createStatement().execute("DROP SEQUENCE s");
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(3, rs.getInt(1));
         conn.close();
@@ -153,7 +93,7 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+20));
         conn = DriverManager.getConnection(getUrl(), props);
         try {
-            rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+            rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
             fail();
         } catch (SequenceNotFoundException e) {
             conn.close();            
@@ -163,14 +103,14 @@ public class PointInTimeQueryIT extends BaseQueryIT {
         conn.close();
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+25));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(1, rs.getInt(1));
         conn.close();
 
         props.put(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts+6));
         conn = DriverManager.getConnection(getUrl(), props);
-        rs = conn.createStatement().executeQuery("SELECT next value for s FROM ATABLE LIMIT 1");
+        rs = conn.createStatement().executeQuery("SELECT next value for s FROM " + tableName + " LIMIT 1");
         assertTrue(rs.next());
         assertEquals(4, rs.getInt(1));
         conn.close();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
index 17c854a..87b7af6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ProductMetricsIT.java
@@ -89,7 +89,7 @@ public class ProductMetricsIT extends BaseClientManagedTimeIT {
     }
     
     private static void initTable(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(), PRODUCT_METRICS_NAME, PRODUCT_METRICS_NAME,splits, ts-2);
+        ensureTableCreated(getUrl(), PRODUCT_METRICS_NAME, PRODUCT_METRICS_NAME,splits, ts-2, null);
     }
 
     private static void assertNoRows(Connection conn) throws SQLException {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
index ec9f32f..6b3778a 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/QueryDatabaseMetaDataIT.java
@@ -655,7 +655,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testCreateDropTable() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         ensureTableCreated(getUrl(), BTABLE_NAME, BTABLE_NAME, ts-2);
         ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-2);
@@ -977,7 +977,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddKVColumnToExistingFamily() throws Throwable {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1008,7 +1008,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddKVColumnToNewFamily() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
         Connection conn1 = DriverManager.getConnection(getUrl(), props);
@@ -1034,7 +1034,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testAddPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1067,7 +1067,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropKVColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1107,7 +1107,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts, getUrl(), null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));
@@ -1125,7 +1125,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     public void testDropAllKVCols() throws Exception {
         ResultSet rs;
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, null, ts);
+        ensureTableCreated(getUrl(), MDTEST_NAME, MDTEST_NAME, null, ts, null);
         
         Properties props = new Properties();
         
@@ -1171,7 +1171,7 @@ public class QueryDatabaseMetaDataIT extends BaseClientManagedTimeIT {
     @Test
     public void testNewerTableDisallowed() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, null, ts);
+        ensureTableCreated(getUrl(), ATABLE_NAME, ATABLE_NAME, null, ts, null);
         
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 5));


[13/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
index f5905ee..d36e0fe 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UpsertSelectIT.java
@@ -89,7 +89,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
     private void testUpsertSelect(boolean createIndex) throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl(), null);
 
         ensureTableCreated(getUrl(), CUSTOM_ENTITY_DATA_FULL_NAME, CUSTOM_ENTITY_DATA_FULL_NAME, ts-1);
         String indexName = "IDX1";
@@ -210,7 +210,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
     public void testUpsertSelectEmptyPKColumn() throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl(), null);
         ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-1);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
@@ -386,7 +386,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
     private void testUpsertSelectForAgg(boolean autoCommit) throws Exception {
         long ts = nextTimestamp();
         String tenantId = getOrganizationId();
-        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl());
+        initATableValues(ATABLE_NAME, tenantId, getDefaultSplits(tenantId), null, ts-1, getUrl(), null);
         ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, ts-1);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
@@ -462,7 +462,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
         byte[][] splits = new byte[][] { PInteger.INSTANCE.toBytes(1), PInteger.INSTANCE.toBytes(2),
                 PInteger.INSTANCE.toBytes(3), PInteger.INSTANCE.toBytes(4)};
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntKeyTest", "IntKeyTest", splits, ts-2);
+        ensureTableCreated(getUrl(),"IntKeyTest", "IntKeyTest", splits, ts-2, null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -602,7 +602,7 @@ public class UpsertSelectIT extends BaseClientManagedTimeIT {
         byte[][] splits = new byte[][] { PInteger.INSTANCE.toBytes(1), PInteger.INSTANCE.toBytes(2),
                 PInteger.INSTANCE.toBytes(3), PInteger.INSTANCE.toBytes(4)};
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntKeyTest", "IntKeyTest", splits,ts-2);
+        ensureTableCreated(getUrl(),"IntKeyTest", "IntKeyTest", splits,ts-2, null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1));
         Connection conn = DriverManager.getConnection(getUrl(), props);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 64935d2..11df167 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;
 
@@ -52,7 +63,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     @Test
     public void testGroupByWithLimitOverRowKey() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,TestUtil.PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,TestUtil.PTSDB_NAME, null, ts-2, null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 10));
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -85,7 +96,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     public void testUpsertDateValues() throws Exception {
         long ts = nextTimestamp();
         Date now = new Date(System.currentTimeMillis());
-        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,TestUtil.PTSDB_NAME,null, ts-2);
+        ensureTableCreated(getUrl(),TestUtil.PTSDB_NAME,TestUtil.PTSDB_NAME,null, ts-2, null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -114,7 +125,7 @@ public class UpsertValuesIT extends BaseClientManagedTimeIT {
     @Test
     public void testUpsertValuesWithExpression() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),"IntKeyTest","IntKeyTest", null, ts-2);
+        ensureTableCreated(getUrl(),"IntKeyTest","IntKeyTest", null, ts-2, null);
         Properties props = new Properties();
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts + 1)); // Execute at timestamp 1
         Connection conn = DriverManager.getConnection(getUrl(), props);
@@ -847,7 +858,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");
@@ -960,6 +971,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/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
index 8b5a591..0b54e73 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/UserDefinedFunctionsIT.java
@@ -141,7 +141,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
                     .append("           throw new ParseException(\"Index cannot be negative :\" + index);\n")
                     .append("        }\n")
                     .append("        Expression arrayExpr = children.get(0);\n")
-                    .append("        return PArrayDataType.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(),getMaxLength());\n")
+                    .append("        return PArrayDataTypeDecoder.positionAtArrayElement(tuple, ptr, index, arrayExpr, getDataType(),getMaxLength());\n")
                     .append("    }\n").toString();
 
     private static String GETY_EVALUATE_METHOD =
@@ -217,6 +217,7 @@ public class UserDefinedFunctionsIT extends BaseOwnClusterIT {
                 .append("import org.apache.phoenix.schema.types.PVarchar;\n")
                 .append("import org.apache.phoenix.util.StringUtil;\n")
                 .append("import org.apache.phoenix.schema.types.PArrayDataType;\n")
+                .append("import org.apache.phoenix.schema.types.PArrayDataTypeDecoder;\n")
                 .append("import org.apache.phoenix.parse.ParseException;\n")
                 .append("public class "+className+" extends ScalarFunction{\n")
                 .append("    public static final String NAME = \""+className+"\";\n")

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
index 6a62673..753f2c8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/VariableLengthPKIT.java
@@ -58,7 +58,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     protected static void initGroupByRowKeyColumns(long ts) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -85,7 +85,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     protected static void initTableValues(byte[][] splits, long ts) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, splits, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, splits, ts-2, null);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -106,7 +106,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
         stmt.setBigDecimal(4, new BigDecimal(.5));
         stmt.execute();
 
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, splits, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, splits, ts-2, null);
         conn.setAutoCommit(false);
 
         // Insert all rows at ts
@@ -431,7 +431,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testNullValueEqualityScan() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -459,7 +459,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testVarLengthPKColScan() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -489,7 +489,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testEscapedQuoteScan() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(), PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -527,7 +527,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     private static void initPtsdbTableValues(long ts) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -560,7 +560,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     }
 
     private static void initPtsdbTableValues2(long ts, Date d) throws Exception {
-        ensureTableCreated(getUrl(),PTSDB2_NAME, PTSDB2_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB2_NAME, PTSDB2_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -696,7 +696,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testBatchUpsert() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB2_NAME, PTSDB2_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB2_NAME, PTSDB2_NAME, null, ts-2, null);
         Date d = new Date(ts);
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(ts));
@@ -874,7 +874,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testMissingPKColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -894,7 +894,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testNoKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -914,7 +914,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     // Broken, since we don't know if insert vs update. @Test
     public void testMissingKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -942,7 +942,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooShortKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -978,7 +978,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooShortPKColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -1014,7 +1014,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooLongPKColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -1051,7 +1051,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testTooLongKVColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),BTABLE_NAME, BTABLE_NAME, null, ts-2, null);
 
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts; // Insert at timestamp 0
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
@@ -1481,7 +1481,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testLikeOnColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -1598,7 +1598,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testILikeOnColumn() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;
@@ -1730,7 +1730,7 @@ public class VariableLengthPKIT extends BaseClientManagedTimeIT {
     @Test
     public void testIsNullInPK() throws Exception {
         long ts = nextTimestamp();
-        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2);
+        ensureTableCreated(getUrl(),PTSDB_NAME, PTSDB_NAME, null, ts-2, null);
 
         // Insert all rows at ts
         String url = getUrl() + ";" + PhoenixRuntime.CURRENT_SCN_ATTRIB + "=" + ts;

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..3d0ba8a 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()));
+                localIndex2.getColumnForColumnName(IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V1")).getColumnQualifierBytes()));
             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.getColumnForColumnName(IndexUtil.getIndexColumnName(QueryConstants.DEFAULT_COLUMN_FAMILY, "V4"));
+            byte[] cq = column.getColumnQualifierBytes();
+            // 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/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
index bc301fa..06802b6 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/ImmutableIndexIT.java
@@ -52,6 +52,7 @@ import org.apache.phoenix.end2end.BaseUniqueNamesOwnClusterIT;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
@@ -70,6 +71,7 @@ import com.google.common.collect.Maps;
 public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
 
     private final boolean localIndex;
+    private final boolean columnEncoded;
     private final String tableDDLOptions;
 
     private volatile boolean stopThreads = false;
@@ -78,9 +80,15 @@ public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
     private static String INDEX_DDL;
     public static final AtomicInteger NUM_ROWS = new AtomicInteger(0);
 
-    public ImmutableIndexIT(boolean localIndex, boolean transactional) {
-        this.localIndex = localIndex;
+    public ImmutableIndexIT(boolean localIndex, boolean transactional, boolean columnEncoded) {
         StringBuilder optionBuilder = new StringBuilder("IMMUTABLE_ROWS=true");
+        this.localIndex = localIndex;
+        this.columnEncoded = columnEncoded;
+        if (!columnEncoded) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0,IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+        }
         if (transactional) {
             optionBuilder.append(", TRANSACTIONAL=true");
         }
@@ -98,11 +106,13 @@ public class ImmutableIndexIT extends BaseUniqueNamesOwnClusterIT {
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
     }
 
-    @Parameters(name="ImmutableIndexIT_localIndex={0},transactional={1}") // name is used by failsafe as file name in reports
+    @Parameters(name="ImmutableIndexIT_localIndex={0},transactional={1},columnEncoded={2}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
 		return Arrays.asList(new Boolean[][] { 
-				{ false, false }, { false, true },
-				{ true, false }, { true, true } });
+				{ false, false, false }, { false, false, true },
+				{ false, true, false }, { false, true, true },
+				{ true, false, false }, { true, false, true },
+                { true, true, false }, { true, true, true } });
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 383452f..2395b02 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
@@ -410,7 +410,12 @@ public class IndexExpressionIT extends ParallelStatsDisabledIT {
             rs = conn.createStatement().executeQuery("SELECT COUNT(*) FROM " + fullIndexTableName);
             assertTrue(rs.next());
             assertEquals(2, rs.getInt(1));
-
+            
+            String sql = "SELECT LONG_COL1 from " + fullDataTableName + " WHERE LONG_COL2 = 2";
+            rs = conn.createStatement().executeQuery(sql);
+            assertTrue(rs.next());
+            assertFalse(rs.next());
+            
             String dml = "DELETE from " + fullDataTableName + " WHERE long_col2 = 2";
             assertEquals(1, conn.createStatement().executeUpdate(dml));
             conn.commit();
@@ -861,8 +866,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());
@@ -1235,7 +1242,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 
         {
@@ -1244,7 +1260,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)");
@@ -1341,7 +1357,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/d6a82e29/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 410dca5..b76d61d 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
@@ -61,8 +61,8 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.NamedTableNode;
 import org.apache.phoenix.parse.TableName;
 import org.apache.phoenix.query.BaseTest;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.util.DateUtil;
@@ -85,27 +85,39 @@ public class IndexIT extends ParallelStatsDisabledIT {
     private final boolean mutable;
     private final String tableDDLOptions;
 
-
-    public IndexIT(boolean localIndex, boolean mutable, boolean transactional) {
+    public IndexIT(boolean localIndex, boolean mutable, boolean transactional, boolean columnEncoded) {
         this.localIndex = localIndex;
         this.transactional = transactional;
         this.mutable = mutable;
         StringBuilder optionBuilder = new StringBuilder();
-        if (!mutable)
-            optionBuilder.append(" IMMUTABLE_ROWS=true ");
+        if (!columnEncoded) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
+        if (!mutable) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
         if (transactional) {
-            if (!(optionBuilder.length()==0))
+            if (optionBuilder.length()!=0)
                 optionBuilder.append(",");
             optionBuilder.append(" TRANSACTIONAL=true ");
         }
         this.tableDDLOptions = optionBuilder.toString();
     }
 
-    @Parameters(name="IndexIT_localIndex={0},mutable={1},transactional={2}") // name is used by failsafe as file name in reports
+    @Parameters(name="IndexIT_localIndex={0},mutable={1},transactional={2},columnEncoded={3}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
         return Arrays.asList(new Boolean[][] {
-                 { false, false, false }, { false, false, true }, { false, true, false }, { false, true, true },
-                 { true, false, false }, { true, false, true }, { true, true, false }, { true, true, true }
+                { false, false, false, false }, { false, false, false, true }, { false, false, true, false }, { false, false, true, true }, 
+                { false, true, false, false }, { false, true, false, true }, { false, true, true, false }, { false, true, true, true }, 
+                { true, false, false, false }, { true, false, false, true }, { true, false, true, false }, { true, false, true, true }, 
+                { true, true, false, false }, { true, true, false, true }, { true, true, true, false }, { true, true, true, true } 
            });
     }
 
@@ -780,7 +792,7 @@ public class IndexIT extends ParallelStatsDisabledIT {
             conn.createStatement().execute(
                     "CREATE TABLE " + testTable
                     + " (k VARCHAR NOT NULL PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) "
-                    + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + "SPLIT ON ('b')");
+                    + (!tableDDLOptions.isEmpty() ? tableDDLOptions : "") + " SPLIT ON ('b')");
             query = "SELECT * FROM " + testTable;
             rs = conn.createStatement().executeQuery(query);
             assertFalse(rs.next());
@@ -808,23 +820,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) {
@@ -897,7 +909,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/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
index e854f23..fb9776e 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/IndexTestUtil.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.Mutation;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.phoenix.hbase.index.util.KeyValueBuilder;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -48,6 +47,7 @@ import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.RowKeySchema;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.types.PDataType;
+import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.SchemaUtil;
@@ -115,7 +115,7 @@ public class IndexTestUtil {
         while ((hasValue = dataRowKeySchema.next(ptr, i, maxOffset)) != null) {
             if (hasValue) {
                 PColumn dataColumn = dataPKColumns.get(i);
-                PColumn indexColumn = indexTable.getColumn(IndexUtil.getIndexColumnName(dataColumn));
+                PColumn indexColumn = indexTable.getColumnForColumnName(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.getColumnForColumnName(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/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
index 60e847e..d07c8fa 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexFailureIT.java
@@ -272,6 +272,8 @@ public class MutableIndexFailureIT extends BaseTest {
             // verify index table has correct data
             validateDataWithIndex(conn, fullTableName, fullIndexName);
             validateDataWithIndex(conn, secondTableName, secondFullIndexName);
+        } finally {
+            FAIL_WRITE = false;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
index 7042fe7..aa9e4eb 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/MutableIndexIT.java
@@ -51,6 +51,7 @@ import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -64,12 +65,17 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     protected final boolean localIndex;
     private final String tableDDLOptions;
 	
-    public MutableIndexIT(boolean localIndex, boolean transactional) {
+    public MutableIndexIT(boolean localIndex, boolean transactional, boolean columnEncoded) {
 		this.localIndex = localIndex;
 		StringBuilder optionBuilder = new StringBuilder();
 		if (transactional) {
 			optionBuilder.append("TRANSACTIONAL=true");
 		}
+		if (!columnEncoded) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
 		this.tableDDLOptions = optionBuilder.toString();
 	}
     
@@ -84,11 +90,13 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
         return getConnection(props);
     }
     
-	@Parameters(name="MutableIndexIT_localIndex={0},transactional={1}") // name is used by failsafe as file name in reports
+	@Parameters(name="MutableIndexIT_localIndex={0},transactional={1},columnEncoded={2}") // name is used by failsafe as file name in reports
     public static Collection<Boolean[]> data() {
-        return Arrays.asList(new Boolean[][] {
-                { false, false }, { false, true }, { true, false }, { true, true }
-           });
+        return Arrays.asList(new Boolean[][] { 
+                { false, false, false }, { false, false, true },
+                { false, true, false }, { false, true, true },
+                { true, false, false }, { true, false, true },
+                { true, true, false }, { true, true, true } });
     }
     
     @Test
@@ -610,11 +618,13 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     }
 
     @Test
+    @Ignore //TODO remove after PHOENIX-3585 is fixed
     public void testSplitDuringIndexScan() throws Exception {
         testSplitDuringIndexScan(false);
     }
     
     @Test
+    @Ignore //TODO remove after PHOENIX-3585 is fixed
     public void testSplitDuringIndexReverseScan() throws Exception {
         testSplitDuringIndexScan(true);
     }
@@ -672,6 +682,7 @@ public class MutableIndexIT extends ParallelStatsDisabledIT {
     }
 
     @Test
+    @Ignore //TODO remove after PHOENIX-3585 is fixed
     public void testIndexHalfStoreFileReader() throws Exception {
         Connection conn1 = getConnection();
         HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(), TestUtil.TEST_PROPERTIES).getAdmin();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
index 29f3758..5ae11bf 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/salted/SaltedTableIT.java
@@ -58,7 +58,7 @@ public class SaltedTableIT extends BaseClientManagedTimeIT {
         // 4abc123jkl444
         try {
             // Upsert with no column specifies.
-            ensureTableCreated(getUrl(), TABLE_WITH_SALTING, TABLE_WITH_SALTING, splits, ts-2);
+            ensureTableCreated(getUrl(), TABLE_WITH_SALTING, TABLE_WITH_SALTING, splits, ts-2, null);
             String query = "UPSERT INTO " + TABLE_WITH_SALTING + " VALUES(?,?,?,?,?)";
             PreparedStatement stmt = conn.prepareStatement(query);
             stmt.setInt(1, 1);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java b/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
new file mode 100644
index 0000000..badf39b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/tx/ParameterizedTransactionIT.java
@@ -0,0 +1,518 @@
+/*
+ * 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.tx;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.util.TestUtil.INDEX_DATA_SCHEMA;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.client.HTableInterface;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.phoenix.coprocessor.PhoenixTransactionalProcessor;
+import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableKey;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.apache.tephra.TxConstants;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Lists;
+
+@RunWith(Parameterized.class)
+public class ParameterizedTransactionIT extends ParallelStatsDisabledIT {
+    
+    private final String tableDDLOptions;
+
+    public ParameterizedTransactionIT(boolean mutable, boolean columnEncoded) {
+        StringBuilder optionBuilder = new StringBuilder("TRANSACTIONAL=true");
+        if (!columnEncoded) {
+            optionBuilder.append(",COLUMN_ENCODED_BYTES=0");
+        }
+        if (!mutable) {
+            optionBuilder.append(",IMMUTABLE_ROWS=true");
+            if (!columnEncoded) {
+                optionBuilder.append(",IMMUTABLE_STORAGE_SCHEME="+PTableImpl.ImmutableStorageScheme.ONE_CELL_PER_COLUMN);
+            }
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+    }
+    
+    @Parameters(name="TransactionIT_mutable={0},columnEncoded={1}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean[]> data() {
+        return Arrays.asList(new Boolean[][] {     
+                 {false, false }, {false, true }, {true, false }, { true, true },
+           });
+    }
+    
+    @Test
+    public void testReadOwnWrites() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        String selectSql = "SELECT * FROM "+ fullTableName;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+            conn.setAutoCommit(false);
+            ResultSet rs = conn.createStatement().executeQuery(selectSql);
+            assertFalse(rs.next());
+            
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(upsert);
+            // upsert two rows
+            TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.execute();
+            TestUtil.setRowKeyColumns(stmt, 2);
+            stmt.execute();
+            
+            // verify rows can be read even though commit has not been called
+            rs = conn.createStatement().executeQuery(selectSql);
+            TestUtil.validateRowKeyColumns(rs, 1);
+            TestUtil.validateRowKeyColumns(rs, 2);
+            assertFalse(rs.next());
+            
+            conn.commit();
+            
+            // verify rows can be read after commit
+            rs = conn.createStatement().executeQuery(selectSql);
+            TestUtil.validateRowKeyColumns(rs, 1);
+            TestUtil.validateRowKeyColumns(rs, 2);
+            assertFalse(rs.next());
+        }
+    }
+    
+    @Test
+    public void testTxnClosedCorrecty() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        String selectSql = "SELECT * FROM "+fullTableName;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+            conn.setAutoCommit(false);
+            ResultSet rs = conn.createStatement().executeQuery(selectSql);
+            assertFalse(rs.next());
+            
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(upsert);
+            // upsert two rows
+            TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.execute();
+            TestUtil.setRowKeyColumns(stmt, 2);
+            stmt.execute();
+            
+            // verify rows can be read even though commit has not been called
+            rs = conn.createStatement().executeQuery(selectSql);
+            TestUtil.validateRowKeyColumns(rs, 1);
+            TestUtil.validateRowKeyColumns(rs, 2);
+            // Long currentTx = rs.unwrap(PhoenixResultSet.class).getCurrentRow().getValue(0).getTimestamp();
+            assertFalse(rs.next());
+            
+            conn.close();
+            // start new connection
+            // conn.createStatement().executeQuery(selectSql);
+            // assertFalse("This transaction should not be on the invalid transactions",
+            // txManager.getCurrentState().getInvalid().contains(currentTx));
+        }
+    }
+    
+    @Test
+    public void testAutoCommitQuerySingleTable() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+            conn.setAutoCommit(true);
+            // verify no rows returned
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName);
+            assertFalse(rs.next());
+        }
+    }
+    
+    @Test
+    public void testAutoCommitQueryMultiTables() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+            conn.setAutoCommit(true);
+            // verify no rows returned
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + fullTableName + " x JOIN " + fullTableName + " y ON (x.long_pk = y.int_pk)");
+            assertFalse(rs.next());
+        } 
+    }
+    
+    @Test
+    public void testSelfJoin() throws Exception {
+        String t1 = generateUniqueName();
+        String t2 = generateUniqueName();
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("create table " + t1 + " (varchar_pk VARCHAR NOT NULL primary key, a.varchar_col1 VARCHAR, b.varchar_col2 VARCHAR)" + tableDDLOptions);
+            conn.createStatement().execute("create table " + t2 + " (varchar_pk VARCHAR NOT NULL primary key, a.varchar_col1 VARCHAR, b.varchar_col1 VARCHAR)" + tableDDLOptions);
+            // verify no rows returned
+            ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + t1 + " x JOIN " + t1 + " y ON (x.varchar_pk = y.a.varchar_col1)");
+            assertFalse(rs.next());
+            rs = conn.createStatement().executeQuery("SELECT * FROM " + t2 + " x JOIN " + t2 + " y ON (x.varchar_pk = y.a.varchar_col1)");
+            assertFalse(rs.next());
+        } 
+    }
+    
+    private void testRowConflicts(String fullTableName) throws Exception {
+        try (Connection conn1 = DriverManager.getConnection(getUrl());
+                Connection conn2 = DriverManager.getConnection(getUrl())) {
+            conn1.setAutoCommit(false);
+            conn2.setAutoCommit(false);
+            String selectSql = "SELECT * FROM "+fullTableName;
+            conn1.setAutoCommit(false);
+            ResultSet rs = conn1.createStatement().executeQuery(selectSql);
+            boolean immutableRows = conn1.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, fullTableName)).isImmutableRows();
+            assertFalse(rs.next());
+            // upsert row using conn1
+            String upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, a.int_col1) VALUES(?, ?, ?, ?, ?, ?, ?)";
+            PreparedStatement stmt = conn1.prepareStatement(upsertSql);
+            TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.setInt(7, 10);
+            stmt.execute();
+            // upsert row using conn2
+            upsertSql = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk, b.int_col2) VALUES(?, ?, ?, ?, ?, ?, ?)";
+            stmt = conn2.prepareStatement(upsertSql);
+            TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.setInt(7, 11);
+            stmt.execute();
+            
+            conn1.commit();
+            //second commit should fail
+            try {
+                conn2.commit();
+                if (!immutableRows) fail();
+            }   
+            catch (SQLException e) {
+                if (immutableRows) fail();
+                assertEquals(e.getErrorCode(), SQLExceptionCode.TRANSACTION_CONFLICT_EXCEPTION.getErrorCode());
+            }
+        }
+    }
+    
+    @Test
+    public void testRowConflictDetected() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+        testRowConflicts(fullTableName);
+    }
+    
+    @Test
+    public void testNoConflictDetectionForImmutableRows() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+        conn.createStatement().execute("ALTER TABLE " + fullTableName + " SET IMMUTABLE_ROWS=true");
+        testRowConflicts(fullTableName);
+    }
+    
+    @Test
+    public void testNonTxToTxTable() throws Exception {
+        String nonTxTableName = generateUniqueName();
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        conn.createStatement().execute("CREATE TABLE " + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)" + tableDDLOptions);
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (1)");
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (2, 'a')");
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (3, 'b')");
+        conn.commit();
+        
+        String index = generateUniqueName();
+        conn.createStatement().execute("CREATE INDEX " + index + " ON " + nonTxTableName + "(v)");
+        // Reset empty column value to an empty value like it is pre-transactions
+        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
+        List<Put>puts = Lists.newArrayList(new Put(PInteger.INSTANCE.toBytes(1)), new Put(PInteger.INSTANCE.toBytes(2)), new Put(PInteger.INSTANCE.toBytes(3)));
+        for (Put put : puts) {
+            put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
+        }
+        htable.put(puts);
+        
+        conn.createStatement().execute("ALTER TABLE " + nonTxTableName + " SET TRANSACTIONAL=true");
+        
+        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes( nonTxTableName));
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes(index));
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (4, 'c')");
+        ResultSet rs = conn.createStatement().executeQuery("SELECT /*+ NO_INDEX */ k FROM " + nonTxTableName + " WHERE v IS NULL");
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null,  nonTxTableName)).isTransactional());
+        assertTrue(rs.next());
+        assertEquals(1,rs.getInt(1));
+        assertFalse(rs.next());
+        conn.commit();
+        
+        conn.createStatement().execute("UPSERT INTO " + nonTxTableName + " VALUES (5, 'd')");
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
+        assertTrue(conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, index)).isTransactional());
+        assertTrue(rs.next());
+        assertEquals(1,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(2,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(3,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(4,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(5,rs.getInt(1));
+        assertFalse(rs.next());
+        conn.rollback();
+        
+        rs = conn.createStatement().executeQuery("SELECT k FROM " + nonTxTableName);
+        assertTrue(rs.next());
+        assertEquals(1,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(2,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(3,rs.getInt(1));
+        assertTrue(rs.next());
+        assertEquals(4,rs.getInt(1));
+        assertFalse(rs.next());
+    }
+    
+    @Ignore
+    @Test
+    public void testNonTxToTxTableFailure() throws Exception {
+        String nonTxTableName = generateUniqueName();
+
+        Connection conn = DriverManager.getConnection(getUrl());
+        // Put table in SYSTEM schema to prevent attempts to update the cache after we disable SYSTEM.CATALOG
+        conn.createStatement().execute("CREATE TABLE SYSTEM." + nonTxTableName + "(k INTEGER PRIMARY KEY, v VARCHAR)" + tableDDLOptions);
+        conn.createStatement().execute("UPSERT INTO SYSTEM." + nonTxTableName + " VALUES (1)");
+        conn.commit();
+        // Reset empty column value to an empty value like it is pre-transactions
+        HTableInterface htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
+        Put put = new Put(PInteger.INSTANCE.toBytes(1));
+        put.add(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES, QueryConstants.EMPTY_COLUMN_BYTES, ByteUtil.EMPTY_BYTE_ARRAY);
+        htable.put(put);
+        
+        HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
+        admin.disableTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+        try {
+            // This will succeed initially in updating the HBase metadata, but then will fail when
+            // the SYSTEM.CATALOG table is attempted to be updated, exercising the code to restore
+            // the coprocessors back to the non transactional ones.
+            conn.createStatement().execute("ALTER TABLE SYSTEM." + nonTxTableName + " SET TRANSACTIONAL=true");
+            fail();
+        } catch (SQLException e) {
+            assertTrue(e.getMessage().contains(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME + " is disabled"));
+        } finally {
+            admin.enableTable(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME);
+            admin.close();
+        }
+        
+        ResultSet rs = conn.createStatement().executeQuery("SELECT k FROM SYSTEM." + nonTxTableName + " WHERE v IS NULL");
+        assertTrue(rs.next());
+        assertEquals(1,rs.getInt(1));
+        assertFalse(rs.next());
+        
+        htable = conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(Bytes.toBytes("SYSTEM." + nonTxTableName));
+        assertFalse(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        assertEquals(1,conn.unwrap(PhoenixConnection.class).getQueryServices().
+                getTableDescriptor(Bytes.toBytes("SYSTEM." + nonTxTableName)).
+                getFamily(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES).getMaxVersions());
+    }
+    
+    @Test
+    public void testCreateTableToBeTransactional() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        String t1 = generateUniqueName();
+        String t2 = generateUniqueName();
+        String ddl = "CREATE TABLE " + t1 + " (k varchar primary key) " + tableDDLOptions;
+        conn.createStatement().execute(ddl);
+        PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
+        PTable table = pconn.getTable(new PTableKey(null, t1));
+        HTableInterface htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
+        assertTrue(table.isTransactional());
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+        
+        try {
+            ddl = "ALTER TABLE " + t1 + " SET transactional=false";
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX.getErrorCode(), e.getErrorCode());
+        }
+
+        HBaseAdmin admin = pconn.getQueryServices().getAdmin();
+        HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(t2));
+        desc.addFamily(new HColumnDescriptor(QueryConstants.DEFAULT_COLUMN_FAMILY_BYTES));
+        admin.createTable(desc);
+        ddl = "CREATE TABLE " + t2 + " (k varchar primary key) transactional=true";
+        conn.createStatement().execute(ddl);
+        assertEquals(Boolean.TRUE.toString(), admin.getTableDescriptor(TableName.valueOf(t2)).getValue(TxConstants.READ_NON_TX_DATA));
+        
+        // Should be ok, as HBase metadata should match existing metadata.
+        ddl = "CREATE TABLE IF NOT EXISTS " + t1 + " (k varchar primary key)"; 
+        try {
+            conn.createStatement().execute(ddl);
+            fail();
+        } catch (SQLException e) {
+            assertEquals(SQLExceptionCode.TX_MAY_NOT_SWITCH_TO_NON_TX.getErrorCode(), e.getErrorCode());
+        }
+        ddl += " transactional=true";
+        conn.createStatement().execute(ddl);
+        table = pconn.getTable(new PTableKey(null, t1));
+        htable = pconn.getQueryServices().getTable(Bytes.toBytes(t1));
+        assertTrue(table.isTransactional());
+        assertTrue(htable.getTableDescriptor().getCoprocessors().contains(PhoenixTransactionalProcessor.class.getName()));
+    }
+
+    @Test
+    public void testCurrentDate() throws Exception {
+        String transTableName = generateUniqueName();
+        String fullTableName = INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + transTableName;
+        String selectSql = "SELECT current_date() FROM "+fullTableName;
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute("create table " + fullTableName + TestUtil.TEST_TABLE_SCHEMA + tableDDLOptions);
+            conn.setAutoCommit(false);
+            ResultSet rs = conn.createStatement().executeQuery(selectSql);
+            assertFalse(rs.next());
+            
+            String upsert = "UPSERT INTO " + fullTableName + "(varchar_pk, char_pk, int_pk, long_pk, decimal_pk, date_pk) VALUES(?, ?, ?, ?, ?, ?)";
+            PreparedStatement stmt = conn.prepareStatement(upsert);
+            // upsert two rows
+            TestUtil.setRowKeyColumns(stmt, 1);
+            stmt.execute();
+            conn.commit();
+            
+            rs = conn.createStatement().executeQuery(selectSql);
+            assertTrue(rs.next());
+            Date date1 = rs.getDate(1);
+            assertFalse(rs.next());
+            
+            Thread.sleep(1000);
+            
+            rs = conn.createStatement().executeQuery(selectSql);
+            assertTrue(rs.next());
+            Date date2 = rs.getDate(1);
+            assertFalse(rs.next());
+            assertTrue("current_date() should change while executing multiple statements", date2.getTime() > date1.getTime());
+        }
+    }
+    
+    
+    @Test
+    public void testParallelUpsertSelect() throws Exception {
+        Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+        props.setProperty(QueryServices.MUTATE_BATCH_SIZE_BYTES_ATTRIB, Integer.toString(512));
+        props.setProperty(QueryServices.SCAN_CACHE_SIZE_ATTRIB, Integer.toString(3));
+        props.setProperty(QueryServices.SCAN_RESULT_CHUNK_SIZE, Integer.toString(3));
+        Connection conn = DriverManager.getConnection(getUrl(), props);
+        conn.setAutoCommit(false);
+        String fullTableName1 = generateUniqueName();
+        String fullTableName2 = generateUniqueName();
+        String sequenceName = "S_" + generateUniqueName();
+        conn.createStatement().execute("CREATE SEQUENCE " + sequenceName);
+        conn.createStatement().execute("CREATE TABLE " + fullTableName1 + " (pk INTEGER PRIMARY KEY, val INTEGER) SALT_BUCKETS=4"
+                + (!tableDDLOptions.isEmpty()? "," : "") + tableDDLOptions);
+        conn.createStatement().execute("CREATE TABLE " + fullTableName2 + " (pk INTEGER PRIMARY KEY, val INTEGER)" + tableDDLOptions);
+
+        for (int i = 0; i < 100; i++) {
+            conn.createStatement().execute("UPSERT INTO " + fullTableName1 + " VALUES (NEXT VALUE FOR " + sequenceName + ", " + (i%10) + ")");
+        }
+        conn.commit();
+        conn.setAutoCommit(true);
+        int upsertCount = conn.createStatement().executeUpdate("UPSERT INTO " + fullTableName2 + " SELECT pk, val FROM " + fullTableName1);
+        assertEquals(100,upsertCount);
+        conn.close();
+    }
+
+    @Test
+    public void testInflightPartialEval() throws SQLException {
+
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String transactTableName = generateUniqueName();
+            Statement stmt = conn.createStatement();
+            stmt.execute("CREATE TABLE " + transactTableName + " (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) " + tableDDLOptions);
+
+            
+            try (Connection conn1 = DriverManager.getConnection(getUrl()); Connection conn2 = DriverManager.getConnection(getUrl())) {
+                conn1.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES ('a','b','x')");
+                // Select to force uncommitted data to be written
+                ResultSet rs = conn1.createStatement().executeQuery("SELECT * FROM " + transactTableName);
+                assertTrue(rs.next());
+                assertEquals("a", rs.getString(1));
+                assertEquals("b", rs.getString(2));
+                assertFalse(rs.next());
+                
+                conn2.createStatement().execute("UPSERT INTO " + transactTableName + " VALUES ('a','c','x')");
+                // Select to force uncommitted data to be written
+                rs = conn2.createStatement().executeQuery("SELECT * FROM " + transactTableName );
+                assertTrue(rs.next());
+                assertEquals("a", rs.getString(1));
+                assertEquals("c", rs.getString(2));
+                assertFalse(rs.next());
+                
+                // If the AndExpression were to see the uncommitted row from conn2, the filter would
+                // filter the row out early and no longer continue to evaluate other cells due to
+                // the way partial evaluation holds state.
+                rs = conn1.createStatement().executeQuery("SELECT * FROM " +  transactTableName + " WHERE v1 != 'c' AND v2 = 'x'");
+                assertTrue(rs.next());
+                assertEquals("a", rs.getString(1));
+                assertEquals("b", rs.getString(2));
+                assertFalse(rs.next());
+                
+                // Same as above for conn1 data
+                rs = conn2.createStatement().executeQuery("SELECT * FROM " + transactTableName + " WHERE v1 != 'b' AND v2 = 'x'");
+                assertTrue(rs.next());
+                assertEquals("a", rs.getString(1));
+                assertEquals("c", rs.getString(2));
+                assertFalse(rs.next());
+            }
+
+        }
+    }
+    
+}


[02/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 4a9cb57..3e2c9b5 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/IndexUtil.java
@@ -18,6 +18,8 @@
 package org.apache.phoenix.util;
 
 import static org.apache.phoenix.query.QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_FAMILY;
+import static org.apache.phoenix.query.QueryConstants.VALUE_COLUMN_QUALIFIER;
 import static org.apache.phoenix.util.PhoenixRuntime.getTable;
 
 import java.io.ByteArrayInputStream;
@@ -28,6 +30,7 @@ import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.ListIterator;
 import java.util.Map;
 
 import org.apache.hadoop.hbase.Cell;
@@ -67,10 +70,12 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
 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.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
 import org.apache.phoenix.expression.visitor.RowKeyExpressionVisitor;
 import org.apache.phoenix.hbase.index.ValueGetter;
 import org.apache.phoenix.hbase.index.covered.update.ColumnReference;
@@ -93,9 +98,12 @@ import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PIndexState;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableType;
 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;
@@ -190,6 +198,11 @@ public class IndexUtil {
                 : QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX + dataColumnFamilyName;
     }
     
+    public static byte[] getLocalIndexColumnFamily(byte[] dataColumnFamilyBytes) {
+        String dataCF = Bytes.toString(dataColumnFamilyBytes);
+        return getLocalIndexColumnFamily(dataCF).getBytes();
+    }
+    
     public static PColumn getDataColumn(PTable dataTable, String indexColumnName) {
         int pos = indexColumnName.indexOf(INDEX_COLUMN_NAME_SEP);
         if (pos < 0) {
@@ -209,7 +222,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);
         }
@@ -236,10 +249,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);
     }
@@ -271,10 +285,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) {
@@ -287,8 +301,6 @@ public class IndexUtil {
                  * updating an existing row.
                  */
                 if (dataMutation instanceof Put) {
-                    // TODO: is this more efficient than looking in our mutation map
-                    // using the key plus finding the PColumn?
                     ValueGetter valueGetter = new ValueGetter() {
                     	
                     	@Override
@@ -303,13 +315,13 @@ public class IndexUtil {
                             if (isEmptyKeyValue(table, ref)) {
                                 return null;
                             }
-                            Map<byte [], List<Cell>> familyMap = dataMutation.getFamilyCellMap();
                             byte[] family = ref.getFamily();
+                            byte[] qualifier = ref.getQualifier();
+                            Map<byte [], List<Cell>> familyMap = dataMutation.getFamilyCellMap();
                             List<Cell> kvs = familyMap.get(family);
                             if (kvs == null) {
                                 return null;
                             }
-                            byte[] qualifier = ref.getQualifier();
                             for (Cell kv : kvs) {
                                 if (Bytes.compareTo(kv.getFamilyArray(), kv.getFamilyOffset(), kv.getFamilyLength(), family, 0, family.length) == 0 &&
                                     Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(), qualifier, 0, qualifier.length) == 0) {
@@ -443,13 +455,19 @@ 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;
+            QualifierEncodingScheme encodingScheme = EncodedColumnsUtil.getQualifierEncodingScheme(scan);
+            Expression[] colExpressions = storeColsInSingleCell ? new SingleCellColumnExpression[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 SingleCellColumnExpression(field, family, qualifier, encodingScheme)
+                            : new KeyValueColumnExpression(field, family, qualifier);
+                colExpressions[i] = dataColumnExpr;
             }
-            return new TupleProjector(keyValueSchema, keyValueColumns);
+            return new TupleProjector(keyValueSchema, colExpressions);
         }
         return null;
     }
@@ -498,8 +516,13 @@ public class IndexUtil {
             ptr.set(indexRowKey, firstCell.getRowOffset() + offset, firstCell.getRowLength() - offset);
             byte[] dataRowKey = indexMaintainer.buildDataRowKey(ptr, viewConstants);
             Get get = new Get(dataRowKey);
+            ImmutableStorageScheme storageScheme = indexMaintainer.getIndexStorageScheme();
             for (int i = 0; i < dataColumns.length; i++) {
-                get.addColumn(dataColumns[i].getFamily(), dataColumns[i].getQualifier());
+                if (storageScheme == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS) {
+                    get.addFamily(dataColumns[i].getFamily());
+                } else {
+                    get.addColumn(dataColumns[i].getFamily(), dataColumns[i].getQualifier());
+                }
             }
             Result joinResult = null;
             if (dataRegion != null) {
@@ -516,7 +539,8 @@ public class IndexUtil {
                     if (table != null) table.close();
                 }
             }
-            
+            // at this point join result has data from the data table. We now need to take this result and
+            // add it to the cells that we are returning. 
             // TODO: handle null case (but shouldn't happen)
             Tuple joinTuple = new ResultTuple(joinResult);
             // This will create a byte[] that captures all of the values from the data table
@@ -524,12 +548,14 @@ public class IndexUtil {
                     tupleProjector.getSchema().toBytes(joinTuple, tupleProjector.getExpressions(),
                         tupleProjector.getValueBitSet(), ptr);
             KeyValue keyValue =
-                    KeyValueUtil.newKeyValue(firstCell.getRowArray(),firstCell.getRowOffset(),firstCell.getRowLength(), TupleProjector.VALUE_COLUMN_FAMILY,
-                        TupleProjector.VALUE_COLUMN_QUALIFIER, firstCell.getTimestamp(), value, 0, value.length);
+                    KeyValueUtil.newKeyValue(firstCell.getRowArray(),firstCell.getRowOffset(),firstCell.getRowLength(), VALUE_COLUMN_FAMILY,
+                        VALUE_COLUMN_QUALIFIER, firstCell.getTimestamp(), value, 0, value.length);
             result.add(keyValue);
         }
-        for (int i = 0; i < result.size(); i++) {
-            final Cell cell = result.get(i);
+        
+        ListIterator<Cell> itr = result.listIterator();
+        while (itr.hasNext()) {
+            final Cell cell = itr.next();
             // TODO: Create DelegateCell class instead
             Cell newCell = new Cell() {
 
@@ -545,7 +571,7 @@ public class IndexUtil {
 
                 @Override
                 public short getRowLength() {
-                    return (short)(cell.getRowLength() - offset);
+                    return (short) (cell.getRowLength() - offset);
                 }
 
                 @Override
@@ -647,8 +673,7 @@ public class IndexUtil {
                     return cell.getRow();
                 }
             };
-            // Wrap cell in cell that offsets row key
-            result.set(i, newCell);
+            itr.set(newCell);
         }
     }
     
@@ -762,4 +787,9 @@ public class IndexUtil {
         }
         return pDataTable;
     }
+    
+    public static boolean isLocalIndexFamily(String family) {
+        return family.indexOf(LOCAL_INDEX_COLUMN_FAMILY_PREFIX) != -1;
+    }
+    
 }

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

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 2b5e622..456f9d4 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -647,4 +647,9 @@ public class MetaDataUtil {
         return Bytes.startsWith(cf, QueryConstants.LOCAL_INDEX_COLUMN_FAMILY_PREFIX_BYTES);
     }
     
+    public static final byte[] getPhysicalTableRowForView(PTable view) {
+        byte[] physicalTableSchemaName = Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(view.getPhysicalName().getString()));
+        byte[] physicalTableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(view.getPhysicalName().getString()));
+        return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 5bfb55d..167a35c 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
@@ -505,9 +505,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.getColumnForColumnName(columnName);
         }
         return getColumnInfo(pColumn);
     }
@@ -1166,9 +1166,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.getColumnForColumnName(columnName);
         }
         return pColumn;
     }
@@ -1206,9 +1206,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.getColumnForColumnName(columnName);
         }
         return pColumn;
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ScanUtil.java
index acaeb31..1fdc73b 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;
@@ -54,6 +55,7 @@ import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.DescVarLengthFastByteComparisons;
 import org.apache.phoenix.filter.BooleanExpressionFilter;
 import org.apache.phoenix.filter.DistinctPrefixFilter;
+import org.apache.phoenix.filter.MultiEncodedCQKeyValueComparisonFilter;
 import org.apache.phoenix.filter.SkipScanFilter;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
@@ -63,6 +65,7 @@ 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;
@@ -266,6 +269,21 @@ public class ScanUtil {
             scan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL,Arrays.asList(filter, andWithFilter)));
         }
     }
+    
+    public static void setQualifierRangesOnFilter(Scan scan, Pair<Integer, Integer> minMaxQualifiers) {
+        Filter filter = scan.getFilter();
+        if (filter != null) {
+            if (filter instanceof FilterList) {
+                for (Filter f : ((FilterList)filter).getFilters()) {
+                    if (f instanceof MultiEncodedCQKeyValueComparisonFilter) {
+                        ((MultiEncodedCQKeyValueComparisonFilter)f).setMinMaxQualifierRange(minMaxQualifiers);
+                    }
+                }
+            } else if (filter instanceof MultiEncodedCQKeyValueComparisonFilter) {
+                ((MultiEncodedCQKeyValueComparisonFilter)filter).setMinMaxQualifierRange(minMaxQualifiers);
+            }
+        }
+    }
 
     public static void setTimeRange(Scan scan, long ts) {
         try {
@@ -894,9 +912,18 @@ public class ScanUtil {
         }
         return true;
     }
+    
+    public static boolean hasDynamicColumns(PTable table) {
+        for (PColumn col : table.getColumns()) {
+            if (col.isDynamic()) {
+                return true;
+            }
+        }
+        return false;
+    }
 
     public static boolean isIndexRebuild(Scan scan) {
         return scan.getAttribute((BaseScannerRegionObserver.REBUILD_INDEXES)) != null;
     }
-
+    
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 ed90426..24e60be 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -47,6 +47,7 @@ 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;
@@ -69,6 +70,8 @@ import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.RowKeySchema;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.RowKeySchema.RowKeySchemaBuilder;
 import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SortOrder;
@@ -155,8 +158,9 @@ public class SchemaUtil {
     			rowSize += KeyValue.getKeyValueDataStructureSize(keyLength, column.getFamilyName().getBytes().length, column.getName().getBytes().length, valueLength);
     		}
     	}
+    	byte[] emptyKeyValueKV = EncodedColumnsUtil.getEmptyKeyValueInfo(table).getFirst();
     	// Empty key value
-    	rowSize += KeyValue.getKeyValueDataStructureSize(keyLength, getEmptyColumnFamily(table).length, QueryConstants.EMPTY_COLUMN_BYTES.length, 0);
+    	rowSize += KeyValue.getKeyValueDataStructureSize(keyLength, getEmptyColumnFamily(table).length, emptyKeyValueKV.length, 0);
     	return rowSize;
     }
     
@@ -410,7 +414,7 @@ public class SchemaUtil {
                 }
             } else {
                 try {
-                    return table.getColumnFamily(familyName.getString()).getColumn(column.getName().getString()).getName().getString();
+                    return table.getColumnFamily(familyName.getString()).getPColumnForColumnName(column.getName().getString()).getName().getString();
                 } catch (ColumnFamilyNotFoundException e) {
                     continue; // Shouldn't happen
                 } catch (ColumnNotFoundException e) {
@@ -615,7 +619,7 @@ public class SchemaUtil {
     
     public static boolean columnExists(PTable table, String columnName) {
         try {
-            table.getColumn(columnName);
+            table.getColumnForColumnName(columnName);
             return true;
         } catch (ColumnNotFoundException e) {
             return false;
@@ -1052,7 +1056,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;
@@ -1073,4 +1077,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/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java b/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
index ac2a850..ade5239 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/cache/TenantCacheTest.java
@@ -47,7 +47,7 @@ public class TenantCacheTest {
         TenantCacheImpl newTenantCache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive);
         ImmutableBytesPtr cacheId = new ImmutableBytesPtr(Bytes.toBytes("a"));
         ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("a"));
-        newTenantCache.addServerCache(cacheId, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory);
+        newTenantCache.addServerCache(cacheId, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true);
         assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
         newTenantCache.removeServerCache(cacheId);
         assertEquals(maxBytes, memoryManager.getAvailableMemory());
@@ -63,7 +63,7 @@ public class TenantCacheTest {
         TenantCacheImpl cache = new TenantCacheImpl(memoryManager, maxServerCacheTimeToLive, ticker);
         ImmutableBytesPtr cacheId1 = new ImmutableBytesPtr(Bytes.toBytes("a"));
         ImmutableBytesWritable cachePtr = new ImmutableBytesWritable(Bytes.toBytes("a"));
-        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory);
+        cache.addServerCache(cacheId1, cachePtr, ByteUtil.EMPTY_BYTE_ARRAY, cacheFactory, true);
         assertEquals(maxBytes-1, memoryManager.getAvailableMemory());
         ticker.time += (maxServerCacheTimeToLive + 1) * 1000000;
         cache.cleanUp();
@@ -91,7 +91,7 @@ public class TenantCacheTest {
         }
 
         @Override
-        public Closeable newCache(ImmutableBytesWritable cachePtr, byte[] txState, MemoryChunk chunk)
+        public Closeable newCache(ImmutableBytesWritable cachePtr, byte[] txState, MemoryChunk chunk, boolean useProtoForIndexMaintainer)
                 throws SQLException {
             return chunk;
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..04b8db3 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.getColumnForColumnName("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/d6a82e29/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 2c4eccc..fa270af 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
@@ -59,6 +59,7 @@ import org.apache.phoenix.expression.aggregator.CountAggregator;
 import org.apache.phoenix.expression.aggregator.ServerAggregators;
 import org.apache.phoenix.expression.function.TimeUnit;
 import org.apache.phoenix.filter.ColumnProjectionFilter;
+import org.apache.phoenix.filter.EncodedQualifiersColumnProjectionFilter;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
@@ -173,7 +174,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")).getColumnForColumnName("K");
             assertTrue(SchemaUtil.isPKColumn(c));
         } finally {
             conn.close();
@@ -923,6 +924,25 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         }
     }
 
+    @Test
+    public void testAggregateOnColumnsNotInGroupByForImmutableEncodedTable() throws Exception {
+        String tableName = generateUniqueName();
+        String ddl = "CREATE IMMUTABLE TABLE  " + tableName +
+                "  (a_string varchar not null, col1 integer, col2 integer" +
+                "  CONSTRAINT pk PRIMARY KEY (a_string))";
+        String query = "SELECT col1, max(a_string) from " + tableName + " group by col2";
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            conn.createStatement().execute(ddl);
+            try {
+                PreparedStatement statement = conn.prepareStatement(query);
+                statement.executeQuery();
+                fail();
+            } catch (SQLException e) { // expected
+                assertEquals(SQLExceptionCode.AGGREGATE_WITH_NOT_GROUP_BY_COLUMN.getErrorCode(), e.getErrorCode());
+            }
+        }
+    }
+
     @Test 
     public void testRegexpSubstrSetScanKeys() throws Exception {
         // First test scan keys are set when the offset is 0 or 1. 
@@ -2322,7 +2342,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         Iterator<Filter> iterator = ScanUtil.getFilterIterator(scan);
         while (iterator.hasNext()) {
             Filter filter = iterator.next();
-            if (filter instanceof ColumnProjectionFilter) {
+            if (filter instanceof EncodedQualifiersColumnProjectionFilter) {
                 return true;
             }
         }
@@ -2624,7 +2644,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         conn.createStatement().execute(ddl);
         PTable table = conn.unwrap(PhoenixConnection.class).getMetaDataCache()
                 .getTableRef(new PTableKey(null,"TABLE_WITH_DEFAULT")).getTable();
-        assertNull(table.getColumn("V").getExpressionStr());
+        assertNull(table.getColumnForColumnName("V").getExpressionStr());
     }
 
     @Test
@@ -2639,7 +2659,7 @@ public class QueryCompilerTest extends BaseConnectionlessQueryTest {
         conn.createStatement().execute(ddl2);
         PTable table = conn.unwrap(PhoenixConnection.class).getMetaDataCache()
                 .getTableRef(new PTableKey(null,"TABLE_WITH_DEFAULT")).getTable();
-        assertNull(table.getColumn("V").getExpressionStr());
+        assertNull(table.getColumnForColumnName("V").getExpressionStr());
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
index 25280fa..6c8ac48 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/QueryOptimizerTest.java
@@ -17,8 +17,13 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.MAX_QUALIFIER;
+import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.MIN_QUALIFIER;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
+import static org.apache.phoenix.query.QueryConstants.ENCODED_EMPTY_COLUMN_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.sql.Array;
@@ -32,9 +37,12 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Properties;
 
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.OrderByCompiler.OrderBy;
 import org.apache.phoenix.jdbc.PhoenixPreparedStatement;
+import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.BaseConnectionlessQueryTest;
 import org.apache.phoenix.query.QueryConstants;
@@ -750,4 +758,47 @@ public class QueryOptimizerTest extends BaseConnectionlessQueryTest {
         return Joiner.on(",").join(pkColsDataTypes);
     }
     
+    @Test
+    public void testMinMaxQualifierRangeWithOrderByOnKVColumn() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = "testMintestMinMaxQualifierRange".toUpperCase();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 VARCHAR) COLUMN_ENCODED_BYTES=4");
+        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
+        ResultSet rs = stmt.executeQuery("SELECT K from " + tableName + " ORDER BY (v1)");
+        assertQualifierRanges(rs, ENCODED_EMPTY_COLUMN_NAME, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+        rs = stmt.executeQuery("SELECT K from " + tableName + " ORDER BY (v1, v2)");
+        assertQualifierRanges(rs, ENCODED_EMPTY_COLUMN_NAME, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+        rs = stmt.executeQuery("SELECT V2 from " + tableName + " ORDER BY (v1)");
+        assertQualifierRanges(rs, ENCODED_EMPTY_COLUMN_NAME, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+        rs = stmt.executeQuery("SELECT V1 from " + tableName + " ORDER BY (v1, v2)");
+        assertQualifierRanges(rs, ENCODED_EMPTY_COLUMN_NAME, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+    }
+    
+    @Test
+    public void testMinMaxQualifierRangeWithNoOrderBy() throws Exception {
+        Connection conn = DriverManager.getConnection(getUrl());
+        String tableName = "testMintestMinMaxQualifierRange".toUpperCase();
+        conn.createStatement().execute("CREATE TABLE " + tableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 VARCHAR) COLUMN_ENCODED_BYTES=4");
+        PhoenixStatement stmt = conn.createStatement().unwrap(PhoenixStatement.class);
+        ResultSet rs = stmt.executeQuery("SELECT K from " + tableName);
+        assertQualifierRanges(rs, ENCODED_CQ_COUNTER_INITIAL_VALUE, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+        rs = stmt.executeQuery("SELECT V2 from " + tableName);
+        assertQualifierRanges(rs, ENCODED_EMPTY_COLUMN_NAME, ENCODED_CQ_COUNTER_INITIAL_VALUE + 1);
+        rs = stmt.executeQuery("SELECT V1 from " + tableName);
+        assertQualifierRanges(rs, ENCODED_EMPTY_COLUMN_NAME, ENCODED_CQ_COUNTER_INITIAL_VALUE);
+    }
+    
+    private static void assertQualifierRanges(ResultSet rs, int minQualifier, int maxQualifier) throws SQLException {
+        Scan scan = rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan().getContext().getScan();
+        assertNotNull(scan.getAttribute(MIN_QUALIFIER));
+        assertNotNull(scan.getAttribute(MAX_QUALIFIER));
+        assertEquals(minQualifier, Bytes.toInt(scan.getAttribute(MIN_QUALIFIER)));
+        assertEquals(maxQualifier, Bytes.toInt(scan.getAttribute(MAX_QUALIFIER)));
+    }
+    
+//    private static void assertQualifierRangesNotPresent(ResultSet rs) throws SQLException {
+//        Scan scan = rs.unwrap(PhoenixResultSet.class).getStatement().getQueryPlan().getContext().getScan();
+//        assertNull(scan.getAttribute(MIN_QUALIFIER));
+//        assertNull(scan.getAttribute(MAX_QUALIFIER));
+//    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/compile/SelectStatementRewriterTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/compile/SelectStatementRewriterTest.java b/phoenix-core/src/test/java/org/apache/phoenix/compile/SelectStatementRewriterTest.java
index f7b6ffd..0f31c2d 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/compile/SelectStatementRewriterTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/compile/SelectStatementRewriterTest.java
@@ -17,10 +17,11 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.TWO_BYTE_QUALIFIERS;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.and;
 import static org.apache.phoenix.util.TestUtil.constantComparison;
-import static org.apache.phoenix.util.TestUtil.multiKVFilter;
+import static org.apache.phoenix.util.TestUtil.multiEncodedKVFilter;
 import static org.apache.phoenix.util.TestUtil.singleKVFilter;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
@@ -79,14 +80,14 @@ public class SelectStatementRewriterTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where organization_id='" + tenantId + "' and a_integer=0 and a_string='foo'";
         Filter filter = compileStatement(query);
         assertEquals(
-                multiKVFilter(and(
+                multiEncodedKVFilter(and(
                         constantComparison(
                             CompareOp.EQUAL,
                             A_INTEGER, 0),
                         constantComparison(
                             CompareOp.EQUAL,
                             A_STRING, "foo")
-                    )),
+                    ), TWO_BYTE_QUALIFIERS),
                 filter);
     }
 
@@ -103,14 +104,14 @@ public class SelectStatementRewriterTest extends BaseConnectionlessQueryTest {
         String query = "select * from atable where a_integer=0 and a_string='foo'";
         Filter filter = compileStatement(query);
         assertEquals(
-                multiKVFilter(and(
+                multiEncodedKVFilter(and(
                         constantComparison(
                             CompareOp.EQUAL,
                             A_INTEGER, 0),
                         constantComparison(
                             CompareOp.EQUAL,
                             A_STRING, "foo")
-                    )),
+                    ), TWO_BYTE_QUALIFIERS),
                 filter);
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 44fc47d..ee23a0b 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
@@ -17,6 +17,7 @@
  */
 package org.apache.phoenix.compile;
 
+import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.TWO_BYTE_QUALIFIERS;
 import static org.apache.phoenix.util.TestUtil.ATABLE_NAME;
 import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
 import static org.apache.phoenix.util.TestUtil.and;
@@ -25,7 +26,7 @@ import static org.apache.phoenix.util.TestUtil.bindParams;
 import static org.apache.phoenix.util.TestUtil.columnComparison;
 import static org.apache.phoenix.util.TestUtil.constantComparison;
 import static org.apache.phoenix.util.TestUtil.in;
-import static org.apache.phoenix.util.TestUtil.multiKVFilter;
+import static org.apache.phoenix.util.TestUtil.multiEncodedKVFilter;
 import static org.apache.phoenix.util.TestUtil.not;
 import static org.apache.phoenix.util.TestUtil.or;
 import static org.apache.phoenix.util.TestUtil.singleKVFilter;
@@ -52,7 +53,6 @@ import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
 import org.apache.hadoop.hbase.filter.Filter;
 import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.util.Bytes;
-import org.apache.phoenix.expression.ColumnExpression;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
@@ -67,8 +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();
-        Expression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getColumn("ID").getPosition()).newColumnExpression();
+        Expression idExpression = new ColumnRef(plan.getTableRef(), plan.getTableRef().getTable().getColumnForColumnName("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().getColumnForColumnName("COMPANY"));
         // FilterList has no equals implementation
         assertTrue(filter instanceof FilterList);
         FilterList filterList = (FilterList)filter;
@@ -148,7 +150,7 @@ 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().getColumnForColumnName("COMPANY");
         assertEquals(
                 singleKVFilter(constantComparison(
                     CompareOp.EQUAL,
@@ -261,10 +263,10 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
         assertEquals(
-            multiKVFilter(columnComparison(
+            multiEncodedKVFilter(columnComparison(
                 CompareOp.EQUAL,
                 A_STRING,
-                B_STRING)),
+                B_STRING), TWO_BYTE_QUALIFIERS),
             filter);
     }
 
@@ -296,7 +298,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         Filter filter = scan.getFilter();
 
         assertEquals(
-            multiKVFilter(and(
+            multiEncodedKVFilter(and(
                 constantComparison(
                     CompareOp.EQUAL,
                     A_INTEGER,
@@ -304,7 +306,7 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
                 constantComparison(
                     CompareOp.EQUAL,
                     A_STRING,
-                    "foo"))),
+                    "foo")), TWO_BYTE_QUALIFIERS),
             filter);
     }
 
@@ -939,17 +941,19 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-
+        PTable table = plan.getTableRef().getTable();
+        Expression aInteger = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+        Expression aString = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_STRING").getPosition()).newColumnExpression();
         assertEquals(
-            multiKVFilter(and(
+            multiEncodedKVFilter(and(
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_INTEGER,
+                    aInteger,
                     0),
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_STRING,
-                    "foo"))),
+                    aString,
+                    "foo")), TWO_BYTE_QUALIFIERS),
             filter);
 
         byte[] startRow = PVarchar.INSTANCE.toBytes(tenantId + tenantTypeId);
@@ -971,17 +975,19 @@ public class WhereCompilerTest extends BaseConnectionlessQueryTest {
         QueryPlan plan = pstmt.optimizeQuery();
         Scan scan = plan.getContext().getScan();
         Filter filter = scan.getFilter();
-
+        PTable table = plan.getTableRef().getTable();
+        Expression aInteger = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+        Expression aString = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_STRING").getPosition()).newColumnExpression();
         assertEquals(
-            multiKVFilter(and(
+            multiEncodedKVFilter(and(
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_INTEGER,
+                    aInteger,
                     0),
                 constantComparison(
                     CompareOp.EQUAL,
-                    A_STRING,
-                    "foo"))),
+                    aString,
+                    "foo")), TWO_BYTE_QUALIFIERS),
             filter);
 
         byte[] startRow = PVarchar.INSTANCE.toBytes(tenantId);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..896fd24 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,16 @@ 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.QualifierEncodingScheme;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
@@ -247,16 +252,17 @@ 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),
+            PName colName = PNameFactory.newName(name);
+            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, colName.getBytes()));
         }
         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, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, 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/d6a82e29/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..df55379 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,19 @@ 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.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableImpl;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.tuple.SingleKeyValueTuple;
 import org.apache.phoenix.schema.tuple.Tuple;
 import org.junit.Test;
@@ -169,15 +174,16 @@ public class LiteralResultIteratorPlanTest {
         for (int i = 0; i < row.length; i++) {
             String name = ParseNodeFactory.createTempAlias();
             Expression expr = LiteralExpression.newConstant(row[i]);
+            PName colName = PNameFactory.newName(name);
             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, colName.getBytes()));
         }
         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, ImmutableStorageScheme.ONE_CELL_PER_COLUMN, QualifierEncodingScheme.NON_ENCODED_QUALIFIERS, 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/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java b/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
index 276d946..8553b73 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/execute/MutationStateTest.java
@@ -127,11 +127,11 @@ public class MutationStateTest {
     private void assertTable(String tableName1,List<KeyValue> keyValues1,String tableName2,List<KeyValue> keyValues2) {
         assertTrue("MUTATION_TEST1".equals(tableName1));
         assertTrue(Bytes.equals(PUnsignedInt.INSTANCE.toBytes(111),CellUtil.cloneRow(keyValues1.get(0))));
-        assertTrue("app1".equals(PVarchar.INSTANCE.toObject(CellUtil.cloneValue(keyValues1.get(0)))));
+        assertTrue("app1".equals(PVarchar.INSTANCE.toObject(CellUtil.cloneValue(keyValues1.get(1)))));
 
         assertTrue("MUTATION_TEST2".equals(tableName2));
         assertTrue(Bytes.equals(PUnsignedInt.INSTANCE.toBytes(222),CellUtil.cloneRow(keyValues2.get(0))));
-        assertTrue("app2".equals(PVarchar.INSTANCE.toObject(CellUtil.cloneValue(keyValues2.get(0)))));
+        assertTrue("app2".equals(PVarchar.INSTANCE.toObject(CellUtil.cloneValue(keyValues2.get(1)))));
 
     }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..195c2f0 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;
@@ -47,6 +48,7 @@ import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.SelectStatement;
 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.RowKeyValueAccessor;
 import org.apache.phoenix.schema.SortOrder;
@@ -117,8 +119,10 @@ 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;
+        PName colName = PNameFactory.newName("ELEM");
+        PColumn elemColumn = new PColumnImpl(PNameFactory.newName("ELEM"), PNameFactory.newName(VALUE_COLUMN_FAMILY), baseType, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, colName.getBytes());
+        colName = PNameFactory.newName("IDX");
+        PColumn indexColumn = withOrdinality ? new PColumnImpl(colName, PNameFactory.newName(VALUE_COLUMN_FAMILY), PInteger.INSTANCE, null, null, true, 0, SortOrder.getDefault(), null, null, false, "", false, false, colName.getBytes()) : 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/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java b/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
index e99a71c..ba36445 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/expression/ArrayConstructorExpressionTest.java
@@ -21,8 +21,11 @@ import static org.junit.Assert.assertArrayEquals;
 
 import java.util.List;
 
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.phoenix.expression.function.ArrayElemRefExpression;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.tuple.Tuple;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.util.ByteUtil;
 import org.junit.Test;
@@ -31,11 +34,18 @@ import com.google.common.collect.Lists;
 
 public class ArrayConstructorExpressionTest {
     
-    private static final byte[] BYTE_ARRAY1 = new byte[]{1,2,3,4,5};
-    private static final byte[] BYTE_ARRAY2 = new byte[]{6,7,8};
-
+    protected static final LiteralExpression CONSTANT_EXPRESSION = LiteralExpression.newConstant(QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
+    protected static final byte[] BYTE_ARRAY1 = new byte[]{1,2,3,4,5};
+    protected static final byte[] BYTE_ARRAY2 = new byte[]{6,7,8};
+    protected Expression FALSE_EVAL_EXPRESSION = new DelegateExpression(LiteralExpression.newConstant(null)) {
+        @Override
+        public boolean evaluate(Tuple tuple, ImmutableBytesWritable ptr) {
+            return false;
+        }
+    };
+    
     @Test
-    public void testArraysWithLeadingNulls() throws Exception {
+    public void testLeadingNulls() throws Exception {
         List<Expression> children = Lists.newArrayListWithExpectedSize(4);
         LiteralExpression nullExpression = LiteralExpression.newConstant(null);
         children.add(nullExpression);
@@ -46,7 +56,6 @@ public class ArrayConstructorExpressionTest {
         ImmutableBytesPtr ptr = new ImmutableBytesPtr();
         
         ArrayElemRefExpression arrayElemRefExpression = new ArrayElemRefExpression(Lists.<Expression>newArrayList(arrayConstructorExpression));
-        
         arrayElemRefExpression.setIndex(1);
         arrayElemRefExpression.evaluate(null, ptr);
         assertArrayEquals(ByteUtil.EMPTY_BYTE_ARRAY, ptr.copyBytesIfNecessary());
@@ -60,4 +69,5 @@ public class ArrayConstructorExpressionTest {
         arrayElemRefExpression.evaluate(null, ptr);
         assertArrayEquals(BYTE_ARRAY2, ptr.copyBytesIfNecessary());
     }
+    
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..2788235 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
@@ -25,13 +25,14 @@ import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
 import java.io.DataOutputStream;
 
-import org.apache.phoenix.schema.types.PBinary;
-import org.apache.phoenix.schema.types.PDecimal;
 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.types.PVarchar;
 import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBinary;
+import org.apache.phoenix.schema.types.PDecimal;
+import org.apache.phoenix.schema.types.PVarchar;
 import org.junit.Test;
 
 public class ColumnExpressionTest {
@@ -40,8 +41,9 @@ public class ColumnExpressionTest {
     public void testSerialization() throws Exception {
         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);
+        PName colName = PNameFactory.newName("c1");
+        PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PDecimal.INSTANCE, maxLen, scale,
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -60,8 +62,9 @@ public class ColumnExpressionTest {
     @Test
     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);
+        PName colName = PNameFactory.newName("c1");
+        PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PBinary.INSTANCE, maxLen, null,
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -80,8 +83,9 @@ public class ColumnExpressionTest {
     @Test
     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);
+        PName colName = PNameFactory.newName("c1");
+        PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PVarchar.INSTANCE, null, scale,
+                true, 20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);
@@ -99,8 +103,9 @@ 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);
+        PName colName = PNameFactory.newName("c1");
+        PColumn column = new PColumnImpl(colName, PNameFactory.newName("f1"), PDecimal.INSTANCE, null, null, true,
+                20, SortOrder.getDefault(), 0, null, false, null, false, false, colName.getBytes());
         ColumnExpression colExp = new KeyValueColumnExpression(column);
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         DataOutputStream dOut = new DataOutputStream(baos);

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 e2cf27d..6aabacf 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
@@ -108,7 +108,7 @@ public class IndexMaintainerTest  extends BaseConnectionlessQueryTest {
             PTable index = pconn.getTable(new PTableKey(pconn.getTenantId(),fullIndexName));
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             table.getIndexMaintainers(ptr, pconn);
-            List<IndexMaintainer> c1 = IndexMaintainer.deserialize(ptr, builder);
+            List<IndexMaintainer> c1 = IndexMaintainer.deserialize(ptr, builder, true);
             assertEquals(1,c1.size());
             IndexMaintainer im1 = c1.get(0);
             
@@ -134,8 +134,7 @@ public class IndexMaintainerTest  extends BaseConnectionlessQueryTest {
             }
             ValueGetter valueGetter = newValueGetter(row, valueMap);
             
-            List<Mutation> indexMutations =
-                    IndexTestUtil.generateIndexData(index, table, dataMutation, ptr, builder);
+            List<Mutation> indexMutations = IndexTestUtil.generateIndexData(index, table, dataMutation, ptr, builder);
             assertEquals(1,indexMutations.size());
             assertTrue(indexMutations.get(0) instanceof Put);
             Mutation indexMutation = indexMutations.get(0);
@@ -310,7 +309,7 @@ public class IndexMaintainerTest  extends BaseConnectionlessQueryTest {
             PTable table = pconn.getTable(new PTableKey(pconn.getTenantId(), "FHA"));
             ImmutableBytesWritable ptr = new ImmutableBytesWritable();
             table.getIndexMaintainers(ptr, pconn);
-            List<IndexMaintainer> indexMaintainerList = IndexMaintainer.deserialize(ptr, GenericKeyValueBuilder.INSTANCE);
+            List<IndexMaintainer> indexMaintainerList = IndexMaintainer.deserialize(ptr, GenericKeyValueBuilder.INSTANCE, true);
             assertEquals(1,indexMaintainerList.size());
             IndexMaintainer indexMaintainer = indexMaintainerList.get(0);
             Set<ColumnReference> indexedColumns = indexMaintainer.getIndexedColumns();

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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..d255e0a 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.getColumnForColumnName("ORGANIZATION_ID").getPosition()).newColumnExpression();
+            ENTITY_ID = new ColumnRef(new TableRef(table), table.getColumnForColumnName("ENTITY_ID").getPosition()).newColumnExpression();
+            A_INTEGER = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_INTEGER").getPosition()).newColumnExpression();
+            A_STRING = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_STRING").getPosition()).newColumnExpression();
+            B_STRING = new ColumnRef(new TableRef(table), table.getColumnForColumnName("B_STRING").getPosition()).newColumnExpression();
+            A_DATE = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_DATE").getPosition()).newColumnExpression();
+            A_TIME = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_TIME").getPosition()).newColumnExpression();
+            A_TIMESTAMP = new ColumnRef(new TableRef(table), table.getColumnForColumnName("A_TIMESTAMP").getPosition()).newColumnExpression();
+            X_DECIMAL = new ColumnRef(new TableRef(table), table.getColumnForColumnName("X_DECIMAL").getPosition()).newColumnExpression();
         } finally {
             conn.close();
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
index c5f4e4f..078c1e8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/query/BaseTest.java
@@ -147,7 +147,8 @@ import org.apache.tephra.TransactionManager;
 import org.apache.tephra.TxConstants;
 import org.apache.tephra.distributed.TransactionService;
 import org.apache.tephra.metrics.TxMetricsCollector;
-import org.apache.tephra.persist.InMemoryTransactionStateStorage;
+import org.apache.tephra.persist.HDFSTransactionStateStorage;
+import org.apache.tephra.snapshot.SnapshotCodecProvider;
 import org.apache.twill.discovery.DiscoveryService;
 import org.apache.twill.discovery.ZKDiscoveryService;
 import org.apache.twill.internal.utils.Networks;
@@ -450,14 +451,18 @@ public abstract class BaseTest {
         
     }
     
-    protected static void setupTxManager() throws SQLException, IOException {
+    protected static void setTxnConfigs() throws IOException {
         config.setBoolean(TxConstants.Manager.CFG_DO_PERSIST, false);
         config.set(TxConstants.Service.CFG_DATA_TX_CLIENT_RETRY_STRATEGY, "n-times");
         config.setInt(TxConstants.Service.CFG_DATA_TX_CLIENT_ATTEMPTS, 1);
         config.setInt(TxConstants.Service.CFG_DATA_TX_BIND_PORT, Networks.getRandomPort());
         config.set(TxConstants.Manager.CFG_TX_SNAPSHOT_DIR, tmpFolder.newFolder().getAbsolutePath());
         config.setInt(TxConstants.Manager.CFG_TX_TIMEOUT, DEFAULT_TXN_TIMEOUT_SECONDS);
-
+        config.unset(TxConstants.Manager.CFG_TX_HDFS_USER);
+        config.setLong(TxConstants.Manager.CFG_TX_SNAPSHOT_INTERVAL, 5L);
+    }
+    
+    protected static void setupTxManager() throws SQLException, IOException {
         ConnectionInfo connInfo = ConnectionInfo.create(getUrl());
         zkClient = ZKClientServices.delegate(
           ZKClients.reWatchOnExpire(
@@ -473,7 +478,7 @@ public abstract class BaseTest {
         zkClient.startAndWait();
 
         DiscoveryService discovery = new ZKDiscoveryService(zkClient);
-        txManager = new TransactionManager(config, new InMemoryTransactionStateStorage(), new TxMetricsCollector());
+        txManager = new TransactionManager(config, new HDFSTransactionStateStorage(config, new SnapshotCodecProvider(config), new TxMetricsCollector()), new TxMetricsCollector());
         txService = new TransactionService(config, zkClient, discovery, Providers.of(txManager));
         txService.startAndWait();
     }
@@ -502,8 +507,9 @@ public abstract class BaseTest {
     /**
      * Set up the test hbase cluster.
      * @return url to be used by clients to connect to the cluster.
+     * @throws IOException 
      */
-    protected static String setUpTestCluster(@Nonnull Configuration conf, ReadOnlyProps overrideProps) {
+    protected static String setUpTestCluster(@Nonnull Configuration conf, ReadOnlyProps overrideProps) throws IOException {
         boolean isDistributedCluster = isDistributedClusterModeEnabled(conf);
         if (!isDistributedCluster) {
             return initMiniCluster(conf, overrideProps);
@@ -558,8 +564,9 @@ public abstract class BaseTest {
     }
     
     protected static void setUpTestDriver(ReadOnlyProps serverProps, ReadOnlyProps clientProps) throws Exception {
+        setTxnConfigs();
         String url = checkClusterInitialized(serverProps);
-        checkTxManagerInitialized(clientProps);
+        checkTxManagerInitialized(serverProps);
         if (driver == null) {
             driver = initAndRegisterTestDriver(url, clientProps);
         }
@@ -713,26 +720,29 @@ public abstract class BaseTest {
     }
 
     protected static void ensureTableCreated(String url, String tableName) throws SQLException {
-        ensureTableCreated(url, tableName, tableName, null, null);
+        ensureTableCreated(url, tableName, tableName, null, null, null);
     }
 
     protected static void ensureTableCreated(String url, String tableName, String tableDDLType) throws SQLException {
-        ensureTableCreated(url, tableName, tableDDLType, null, null);
+        ensureTableCreated(url, tableName, tableDDLType, null, null, null);
     }
 
-    public static void ensureTableCreated(String url, String tableName, String tableDDLType, byte[][] splits) throws SQLException {
-        ensureTableCreated(url, tableName, tableDDLType, splits, null);
+    public static void ensureTableCreated(String url, String tableName, String tableDDLType, byte[][] splits, String tableDDLOptions) throws SQLException {
+        ensureTableCreated(url, tableName, tableDDLType, splits, null, tableDDLOptions);
     }
 
     protected static void ensureTableCreated(String url, String tableName, String tableDDLType, Long ts) throws SQLException {
-        ensureTableCreated(url, tableName, tableDDLType, null, ts);
+        ensureTableCreated(url, tableName, tableDDLType, null, ts, null);
     }
 
-    protected static void ensureTableCreated(String url, String tableName, String tableDDLType, byte[][] splits, Long ts) throws SQLException {
+    protected static void ensureTableCreated(String url, String tableName, String tableDDLType, byte[][] splits, Long ts, String tableDDLOptions) throws SQLException {
         String ddl = tableDDLMap.get(tableDDLType);
         if(!tableDDLType.equals(tableName)) {
            ddl =  ddl.replace(tableDDLType, tableName);
         }
+        if (tableDDLOptions!=null) {
+            ddl += tableDDLOptions;
+        }
         createSchema(url,tableName, ts);
         createTestTable(url, ddl, splits, ts);
     }
@@ -977,7 +987,7 @@ public abstract class BaseTest {
     }
 
     protected static void initSumDoubleValues(String tableName, byte[][] splits, String url) throws Exception {
-        ensureTableCreated(url, tableName, SUM_DOUBLE_NAME, splits);
+        ensureTableCreated(url, tableName, SUM_DOUBLE_NAME, splits, null);
         Properties props = new Properties();
         Connection conn = DriverManager.getConnection(url, props);
         try {
@@ -1045,26 +1055,26 @@ public abstract class BaseTest {
     }
 
     protected static String initATableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
-        return initATableValues(null, tenantId, splits, date, ts, url);
+        return initATableValues(null, tenantId, splits, date, ts, url, null);
     }
     
-    protected static String initATableValues(String tableName, String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
+    protected static String initATableValues(String tableName, String tenantId, byte[][] splits, Date date, Long ts, String url, String tableDDLOptions) throws Exception {
         if(tableName == null) {
             tableName = generateUniqueName();
         }
         String tableDDLType = ATABLE_NAME;
         if (ts == null) {
-            ensureTableCreated(url, tableName, tableDDLType, splits);
+            ensureTableCreated(url, tableName, tableDDLType, splits, null, tableDDLOptions);
         } else {
-            ensureTableCreated(url, tableName, tableDDLType, splits, ts-5);
+            ensureTableCreated(url, tableName, tableDDLType, splits, ts-5, tableDDLOptions);
         }
         
         Properties props = new Properties();
         if (ts != null) {
             props.setProperty(CURRENT_SCN_ATTRIB, Long.toString(ts-3));
         }
-        Connection conn = DriverManager.getConnection(url, props);
-        try {
+        
+        try (Connection conn = DriverManager.getConnection(url, props)) {
             // Insert all rows at ts
             PreparedStatement stmt = conn.prepareStatement(
                     "upsert into " + tableName +
@@ -1251,12 +1261,9 @@ public abstract class BaseTest {
             stmt.setFloat(15, 0.09f);
             stmt.setDouble(16, 0.0009);
             stmt.execute();
-                
             conn.commit();
-        } finally {
-            conn.close();
-            return tableName;
         }
+        return tableName;
     }
 
     
@@ -1278,9 +1285,9 @@ public abstract class BaseTest {
     
     private static void initEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
         if (ts == null) {
-            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, ENTITY_HISTORY_TABLE_NAME, splits);
+            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, ENTITY_HISTORY_TABLE_NAME, splits, null);
         } else {
-            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, ENTITY_HISTORY_TABLE_NAME, splits, ts-2);
+            ensureTableCreated(url, ENTITY_HISTORY_TABLE_NAME, ENTITY_HISTORY_TABLE_NAME, splits, ts-2, null);
         }
         
         Properties props = new Properties();
@@ -1382,9 +1389,9 @@ public abstract class BaseTest {
     
     protected static void initSaltedEntityHistoryTableValues(String tenantId, byte[][] splits, Date date, Long ts, String url) throws Exception {
         if (ts == null) {
-            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, ENTITY_HISTORY_SALTED_TABLE_NAME, splits);
+            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, null);
         } else {
-            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, ts-2);
+            ensureTableCreated(url, ENTITY_HISTORY_SALTED_TABLE_NAME, ENTITY_HISTORY_SALTED_TABLE_NAME, splits, ts-2, null);
         }
         
         Properties props = new Properties();


[06/17] phoenix git commit: PHOENIX-1598 Encode column names to save space and improve performance(Samarth Jain and Thomas D'Silva)

Posted by sa...@apache.org.
http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 e499e29..9f944c7 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
@@ -70,7 +70,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/d6a82e29/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 51587f1..d563bc2 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,17 @@
  */
 package org.apache.phoenix.iterate;
 
+import static com.google.common.base.Preconditions.checkArgument;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_ACTUAL_START_ROW;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_START_ROW_SUFFIX;
 import static org.apache.phoenix.coprocessor.BaseScannerRegionObserver.SCAN_STOP_ROW_SUFFIX;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_FAILED_QUERY_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_TIMEOUT_COUNTER;
+import static org.apache.phoenix.schema.PTable.IndexType.LOCAL;
+import static org.apache.phoenix.schema.PTableType.INDEX;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
+import static org.apache.phoenix.util.EncodedColumnsUtil.isPossibleToUseEncodedCQFilter;
+import static org.apache.phoenix.util.ScanUtil.hasDynamicColumns;
 
 import java.io.ByteArrayInputStream;
 import java.io.DataInput;
@@ -30,6 +35,7 @@ import java.io.DataInputStream;
 import java.io.EOFException;
 import java.sql.SQLException;
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -69,6 +75,7 @@ import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
 import org.apache.phoenix.filter.ColumnProjectionFilter;
 import org.apache.phoenix.filter.DistinctPrefixFilter;
+import org.apache.phoenix.filter.EncodedQualifiersColumnProjectionFilter;
 import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
 import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.parse.FilterableStatement;
@@ -76,12 +83,13 @@ import org.apache.phoenix.parse.HintNode;
 import org.apache.phoenix.parse.HintNode.Hint;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
-import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.schema.PTable.IndexType;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.StaleRegionBoundaryCacheException;
 import org.apache.phoenix.schema.TableRef;
@@ -90,6 +98,8 @@ import org.apache.phoenix.schema.stats.GuidePostsKey;
 import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.Closeables;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.LogUtil;
 import org.apache.phoenix.util.PrefixByteCodec;
 import org.apache.phoenix.util.PrefixByteDecoder;
@@ -157,7 +167,7 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
         return true;
     }
     
-    private static void initializeScan(QueryPlan plan, Integer perScanLimit, Integer offset, Scan scan) {
+    private static void initializeScan(QueryPlan plan, Integer perScanLimit, Integer offset, Scan scan) throws SQLException {
         StatementContext context = plan.getContext();
         TableRef tableRef = plan.getTableRef();
         PTable table = tableRef.getTable();
@@ -208,7 +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) &&
@@ -241,13 +250,93 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                     ScanUtil.andFilterAtEnd(scan, new PageFilter(plan.getLimit()));
                 }
             }
-
+            scan.setAttribute(BaseScannerRegionObserver.QUALIFIER_ENCODING_SCHEME, new byte[]{table.getEncodingScheme().getSerializedMetadataValue()});
+            scan.setAttribute(BaseScannerRegionObserver.IMMUTABLE_STORAGE_ENCODING_SCHEME, new byte[]{table.getImmutableStorageScheme().getSerializedMetadataValue()});
+            // we use this flag on the server side to determine which value column qualifier to use in the key value we return from server.
+            scan.setAttribute(BaseScannerRegionObserver.USE_NEW_VALUE_COLUMN_QUALIFIER, Bytes.toBytes(true));
+            // When analyzing the table, there is no look up for key values being done.
+            // So there is no point setting the range.
+            if (EncodedColumnsUtil.setQualifierRanges(table) && !ScanUtil.isAnalyzeTable(scan)) {
+                Pair<Integer, Integer> range = getEncodedQualifierRange(scan, context);
+                if (range != null) {
+                    scan.setAttribute(BaseScannerRegionObserver.MIN_QUALIFIER, Bytes.toBytes(range.getFirst()));
+                    scan.setAttribute(BaseScannerRegionObserver.MAX_QUALIFIER, Bytes.toBytes(range.getSecond()));
+                    ScanUtil.setQualifierRangesOnFilter(scan, range);
+                }
+            }
             if (optimizeProjection) {
                 optimizeProjection(context, scan, table, statement);
             }
         }
     }
+    
+    private static Pair<Integer, Integer> getEncodedQualifierRange(Scan scan, StatementContext context)
+            throws SQLException {
+        PTable table = context.getCurrentTable().getTable();
+        QualifierEncodingScheme encodingScheme = table.getEncodingScheme();
+        checkArgument(encodingScheme != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS,
+            "Method should only be used for tables using encoded column names");
+        Pair<Integer, Integer> minMaxQualifiers = new Pair<>();
+        for (Pair<byte[], byte[]> whereCol : context.getWhereConditionColumns()) {
+            byte[] cq = whereCol.getSecond();
+            if (cq != null) {
+                int qualifier = table.getEncodingScheme().decode(cq);
+                determineQualifierRange(qualifier, minMaxQualifiers);
+            }
+        }
+        Map<byte[], NavigableSet<byte[]>> familyMap = scan.getFamilyMap();
+
+        Map<String, Pair<Integer, Integer>> qualifierRanges = EncodedColumnsUtil.getFamilyQualifierRanges(table);
+        for (Entry<byte[], NavigableSet<byte[]>> entry : familyMap.entrySet()) {
+            if (entry.getValue() != null) {
+                for (byte[] cq : entry.getValue()) {
+                    if (cq != null) {
+                        int qualifier = table.getEncodingScheme().decode(cq);
+                        determineQualifierRange(qualifier, minMaxQualifiers);
+                    }
+                }
+            } else {
+                /*
+                 * All the columns of the column family are being projected. So we will need to
+                 * consider all the columns in the column family to determine the min-max range.
+                 */
+                String family = Bytes.toString(entry.getKey());
+                if (table.getType() == INDEX && table.getIndexType() == LOCAL && !IndexUtil.isLocalIndexFamily(family)) {
+                    //TODO: samarth confirm with James why do we need this hack here :(
+                    family = IndexUtil.getLocalIndexColumnFamily(family);
+                }
+                Pair<Integer, Integer> range = qualifierRanges.get(family);
+                if (range != null) {
+                    determineQualifierRange(range.getFirst(), minMaxQualifiers);
+                    determineQualifierRange(range.getSecond(), minMaxQualifiers);
+                }
+            }
+        }
+        if (minMaxQualifiers.getFirst() == null) {
+            return null;
+        }
+        return minMaxQualifiers;
+    }
 
+    /**
+     * 
+     * @param cq
+     * @param minMaxQualifiers
+     * @return true if the empty column was projected
+     */
+    private static void determineQualifierRange(Integer qualifier, Pair<Integer, Integer> minMaxQualifiers) {
+        if (minMaxQualifiers.getFirst() == null) {
+            minMaxQualifiers.setFirst(qualifier);
+            minMaxQualifiers.setSecond(qualifier);
+        } else {
+            if (minMaxQualifiers.getFirst() > qualifier) {
+                minMaxQualifiers.setFirst(qualifier);
+            } else if (minMaxQualifiers.getSecond() < qualifier) {
+                minMaxQualifiers.setSecond(qualifier);
+            }
+        }
+    }
+    
     private static void optimizeProjection(StatementContext context, Scan scan, PTable table, FilterableStatement statement) {
         Map<byte[], NavigableSet<byte[]>> familyMap = scan.getFamilyMap();
         // columnsTracker contain cf -> qualifiers which should get returned.
@@ -255,6 +344,9 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                 new TreeMap<ImmutableBytesPtr, NavigableSet<ImmutableBytesPtr>>();
         Set<byte[]> conditionOnlyCfs = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
         int referencedCfCount = familyMap.size();
+        QualifierEncodingScheme encodingScheme = table.getEncodingScheme();
+        ImmutableStorageScheme storageScheme = table.getImmutableStorageScheme();
+        BitSet trackedColumnsBitset = isPossibleToUseEncodedCQFilter(encodingScheme, storageScheme) && !hasDynamicColumns(table) ? new BitSet(10) : null;
         boolean filteredColumnNotInProjection = false;
         for (Pair<byte[], byte[]> whereCol : context.getWhereConditionColumns()) {
             byte[] filteredFamily = whereCol.getFirst();
@@ -295,6 +387,10 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
                 cols = new TreeSet<ImmutableBytesPtr>();
                 for (byte[] q : qs) {
                     cols.add(new ImmutableBytesPtr(q));
+                    if (trackedColumnsBitset != null) {
+                        int qualifier = encodingScheme.decode(q);
+                        trackedColumnsBitset.set(qualifier);
+                    }
                 }
             }
             columnsTracker.put(cf, cols);
@@ -343,8 +439,9 @@ public abstract class BaseResultIterators extends ExplainTable implements Result
             // in the scan in this case. We still want the other optimization that causes
             // the ExplicitColumnTracker not to be used, though.
             if (!statement.isAggregate() && filteredColumnNotInProjection) {
-                ScanUtil.andFilterAtEnd(scan, new ColumnProjectionFilter(SchemaUtil.getEmptyColumnFamily(table),
-                        columnsTracker, conditionOnlyCfs));
+                ScanUtil.andFilterAtEnd(scan, 
+                        trackedColumnsBitset != null ? new EncodedQualifiersColumnProjectionFilter(SchemaUtil.getEmptyColumnFamily(table), trackedColumnsBitset, conditionOnlyCfs, table.getEncodingScheme()) : new ColumnProjectionFilter(SchemaUtil.getEmptyColumnFamily(table),
+                        columnsTracker, conditionOnlyCfs, EncodedColumnsUtil.usesEncodedColumnNames(table.getEncodingScheme())));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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/d6a82e29/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/d6a82e29/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 da75bb7..5afdfea 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/d6a82e29/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 52fbe9c..a5a40e2 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,27 @@ 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.PTable.QualifierEncodingScheme;
+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.EncodedColumnsUtil;
 import org.apache.phoenix.util.ServerUtil;
 
 
 public class RegionScannerResultIterator extends BaseResultIterator {
     private final RegionScanner scanner;
-
-    public RegionScannerResultIterator(RegionScanner scanner) {
+    private final Pair<Integer, Integer> minMaxQualifiers;
+    private final boolean useQualifierAsIndex;
+    private final QualifierEncodingScheme encodingScheme;
+    
+    public RegionScannerResultIterator(RegionScanner scanner, Pair<Integer, Integer> minMaxQualifiers, QualifierEncodingScheme encodingScheme) {
         this.scanner = scanner;
+        this.useQualifierAsIndex = EncodedColumnsUtil.useQualifierAsIndex(minMaxQualifiers);
+        this.minMaxQualifiers = minMaxQualifiers;
+        this.encodingScheme = encodingScheme;
     }
 
     @Override
@@ -43,7 +54,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(), encodingScheme) :  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
@@ -54,7 +65,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/d6a82e29/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 5142b57..0e62164 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;
 
@@ -322,6 +322,15 @@ 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 IMMUTABLE_STORAGE_SCHEME = "IMMUTABLE_STORAGE_SCHEME";
+    public static final byte[] STORAGE_SCHEME_BYTES = Bytes.toBytes(IMMUTABLE_STORAGE_SCHEME);
+    public static final String ENCODING_SCHEME = "ENCODING_SCHEME";
+    public static final byte[] ENCODING_SCHEME_BYTES = Bytes.toBytes(ENCODING_SCHEME);
+    public static final String COLUMN_QUALIFIER = "COLUMN_QUALIFIER";
+    public static final byte[] COLUMN_QUALIFIER_BYTES = Bytes.toBytes(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;
@@ -595,9 +604,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/d6a82e29/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/d6a82e29/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 3072736..a8ddd62 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
@@ -56,7 +56,7 @@ public class HashCacheFactory implements ServerCacheFactory {
     }
 
     @Override
-    public Closeable newCache(ImmutableBytesWritable cachePtr, byte[] txState, MemoryChunk chunk) throws SQLException {
+    public Closeable newCache(ImmutableBytesWritable cachePtr, byte[] txState, MemoryChunk chunk, boolean useProtoForIndexMaintainer) throws SQLException {
         try {
             // This reads the uncompressed length from the front of the compressed input
             int uncompressedLen = Snappy.getUncompressedLength(cachePtr.get(), cachePtr.getOffset());

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..278489d 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,22 @@ 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 = c.getColumnQualifierBytes();
+                } else {
+                    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 +248,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);
         }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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..c529afe 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 = c.getColumnQualifierBytes();
+                } 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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
index 47a38a7..54dc748 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/mapreduce/index/PhoenixIndexPartialBuildMapper.java
@@ -113,7 +113,7 @@ public class PhoenixIndexPartialBuildMapper extends TableMapper<ImmutableBytesWr
                     if (put == null) {
                         put = new Put(CellUtil.cloneRow(cell));
                         put.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                        put.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+                        put.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
                         put.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS, PDataType.TRUE_BYTES);
                         mutations.add(put);
                     }
@@ -122,7 +122,7 @@ public class PhoenixIndexPartialBuildMapper extends TableMapper<ImmutableBytesWr
                     if (del == null) {
                         del = new Delete(CellUtil.cloneRow(cell));
                         del.setAttribute(PhoenixIndexCodec.INDEX_UUID, uuidValue);
-                        del.setAttribute(PhoenixIndexCodec.INDEX_MD, attribValue);
+                        del.setAttribute(PhoenixIndexCodec.INDEX_PROTO_MD, attribValue);
                         del.setAttribute(BaseScannerRegionObserver.IGNORE_NEWER_MUTATIONS, PDataType.TRUE_BYTES);
                         mutations.add(del);
                     }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 7777b63..25c7e6e 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
@@ -23,11 +23,34 @@ import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MAJOR_VERS
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_MINOR_VERSION;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.PHOENIX_PATCH_NUMBER;
 import static org.apache.phoenix.coprocessor.MetaDataProtocol.getVersion;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ARRAY_SIZE;
+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_SIZE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE;
+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.IS_ROW_TIMESTAMP;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IS_VIEW_REFERENCED;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.KEY_SEQ;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.NULLABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_STATS_NAME;
+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.TENANT_ID;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_CONSTANT;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_HCONNECTIONS_COUNTER;
 import static org.apache.phoenix.monitoring.GlobalClientMetrics.GLOBAL_QUERY_SERVICES_COUNTER;
+import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_ENABLED;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RENEW_LEASE_THREAD_POOL_SIZE;
@@ -41,6 +64,7 @@ import java.lang.ref.WeakReference;
 import java.sql.PreparedStatement;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -160,11 +184,13 @@ import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
 import org.apache.phoenix.schema.EmptySequenceCacheException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
+import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.NewerSchemaAlreadyExistsException;
 import org.apache.phoenix.schema.NewerTableAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PColumnImpl;
 import org.apache.phoenix.schema.PMetaData;
 import org.apache.phoenix.schema.PMetaDataImpl;
 import org.apache.phoenix.schema.PName;
@@ -178,6 +204,7 @@ import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.Sequence;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceKey;
+import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableAlreadyExistsException;
 import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.schema.TableProperty;
@@ -187,7 +214,9 @@ import org.apache.phoenix.schema.types.PBoolean;
 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.PTinyint;
 import org.apache.phoenix.schema.types.PUnsignedTinyint;
+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.Closeables;
@@ -588,7 +617,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             latestMetaDataLock.notifyAll();
         }
     }
-
     @Override
     public void updateResolvedTimestamp(PTable table, long resolvedTime) throws SQLException {
         synchronized (latestMetaDataLock) {
@@ -2708,6 +2736,31 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                             MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_9_0);
                     clearCache();
                 }
+                if (currentServerSideTableTimeStamp < MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0) {
+                    metaConnection = addColumnQualifierColumn(metaConnection, MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 3);
+                    metaConnection = addColumnsIfNotExists(
+                            metaConnection,
+                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 2,
+                            PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME + " "
+                                    + PTinyint.INSTANCE.getSqlTypeName());
+                    metaConnection = addColumnsIfNotExists(
+                            metaConnection,
+                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0 - 1,
+                            PhoenixDatabaseMetaData.ENCODING_SCHEME + " "
+                                    + PTinyint.INSTANCE.getSqlTypeName());
+                    metaConnection = addColumnsIfNotExists(
+                            metaConnection,
+                            PhoenixDatabaseMetaData.SYSTEM_CATALOG,
+                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0,
+                            PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER + " "
+                                    + PInteger.INSTANCE.getSqlTypeName());
+                    ConnectionQueryServicesImpl.this.removeTable(null,
+                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME, null,
+                            MetaDataProtocol.MIN_SYSTEM_TABLE_TIMESTAMP_4_10_0);
+                    clearCache();
+                }
             }
 
 
@@ -2844,6 +2897,84 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
         }
     }
+    
+    // Special method for adding the column qualifier column for 4.10. 
+    private PhoenixConnection addColumnQualifierColumn(PhoenixConnection oldMetaConnection, Long timestamp) throws SQLException {
+        Properties props = PropertiesUtil.deepCopy(oldMetaConnection.getClientInfo());
+        props.setProperty(PhoenixRuntime.CURRENT_SCN_ATTRIB, Long.toString(timestamp));
+        // Cannot go through DriverManager or you end up in an infinite loop because it'll call init again
+        PhoenixConnection metaConnection = new PhoenixConnection(oldMetaConnection, this, props);
+        PTable sysCatalogPTable = metaConnection.getTable(new PTableKey(null, PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME));
+        int numColumns = sysCatalogPTable.getColumns().size();
+        try (PreparedStatement mutateTable = metaConnection.prepareStatement(MetaDataClient.MUTATE_TABLE)) {
+            mutateTable.setString(1, null);
+            mutateTable.setString(2, SYSTEM_CATALOG_SCHEMA);
+            mutateTable.setString(3, SYSTEM_CATALOG_TABLE);
+            mutateTable.setString(4, PTableType.SYSTEM.getSerializedValue());
+            mutateTable.setLong(5, sysCatalogPTable.getSequenceNumber() + 1);
+            mutateTable.setInt(6, numColumns + 1);
+            mutateTable.execute();
+        }
+        List<Mutation> tableMetadata = new ArrayList<>();
+        tableMetadata.addAll(metaConnection.getMutationState().toMutations(metaConnection.getSCN()).next().getSecond());
+        metaConnection.rollback();
+        PColumn column = new PColumnImpl(PNameFactory.newName("COLUMN_QUALIFIER"),
+                PNameFactory.newName(DEFAULT_COLUMN_FAMILY_NAME), PVarbinary.INSTANCE, null, null, true, numColumns,
+                SortOrder.ASC, null, null, false, null, false, false, 
+                Bytes.toBytes("COLUMN_QUALIFIER"));
+        String upsertColumnMetadata = "UPSERT INTO " + SYSTEM_CATALOG_SCHEMA + ".\"" + SYSTEM_CATALOG_TABLE + "\"( " +
+                TENANT_ID + "," +
+                TABLE_SCHEM + "," +
+                TABLE_NAME + "," +
+                COLUMN_NAME + "," +
+                COLUMN_FAMILY + "," +
+                DATA_TYPE + "," +
+                NULLABLE + "," +
+                COLUMN_SIZE + "," +
+                DECIMAL_DIGITS + "," +
+                ORDINAL_POSITION + "," +
+                SORT_ORDER + "," +
+                DATA_TABLE_NAME + "," +
+                ARRAY_SIZE + "," +
+                VIEW_CONSTANT + "," +
+                IS_VIEW_REFERENCED + "," +
+                PK_NAME + "," +
+                KEY_SEQ + "," +
+                COLUMN_DEF + "," +
+                IS_ROW_TIMESTAMP +
+                ") VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?)";
+        try (PreparedStatement colUpsert = metaConnection.prepareStatement(upsertColumnMetadata)) {
+            colUpsert.setString(1, null);
+            colUpsert.setString(2, SYSTEM_CATALOG_SCHEMA);
+            colUpsert.setString(3, SYSTEM_CATALOG_TABLE);
+            colUpsert.setString(4, "COLUMN_QUALIFIER");
+            colUpsert.setString(5, DEFAULT_COLUMN_FAMILY);
+            colUpsert.setInt(6, column.getDataType().getSqlType());
+            colUpsert.setInt(7, ResultSetMetaData.columnNullable);
+            colUpsert.setNull(8, Types.INTEGER);
+            colUpsert.setNull(9, Types.INTEGER);
+            colUpsert.setInt(10, sysCatalogPTable.getBucketNum() != null ? numColumns : (numColumns + 1));
+            colUpsert.setInt(11, SortOrder.ASC.getSystemValue());
+            colUpsert.setString(12, null);
+            colUpsert.setNull(13, Types.INTEGER);
+            colUpsert.setBytes(14, null);
+            colUpsert.setBoolean(15, false);
+            colUpsert.setString(16, sysCatalogPTable.getPKName() == null ? null : sysCatalogPTable.getPKName().getString());
+            colUpsert.setNull(17, Types.SMALLINT);
+            colUpsert.setNull(18, Types.VARCHAR);
+            colUpsert.setBoolean(19, false);
+            colUpsert.execute();
+        }
+        tableMetadata.addAll(metaConnection.getMutationState().toMutations(metaConnection.getSCN()).next().getSecond());
+        metaConnection.rollback();
+        metaConnection.getQueryServices().addColumn(tableMetadata, sysCatalogPTable, Collections.<String,List<Pair<String,Object>>>emptyMap(), Collections.<String>emptySet(), Lists.newArrayList(column));
+        metaConnection.removeTable(null, SYSTEM_CATALOG_NAME, null, timestamp);
+        ConnectionQueryServicesImpl.this.removeTable(null,
+                SYSTEM_CATALOG_NAME, null,
+                timestamp);
+        clearCache();
+        return metaConnection;
+    }
 
     private void createSnapshot(String snapshotName, String tableName)
             throws SQLException {
@@ -4137,4 +4268,4 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     public Configuration getConfiguration() {
         return config;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 7c78083..47ef954 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/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryConstants.java
index 8e2dc1a..6f105f1 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,8 @@ 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;
+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,11 +42,13 @@ 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.ENCODING_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FUNCTION_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_ROW_COUNT;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POSTS_WIDTH;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.GUIDE_POST_KEY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_STORAGE_SCHEME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INCREMENT_BY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE;
@@ -117,8 +121,8 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.SortOrder;
-import org.apache.phoenix.util.ByteUtil;
 
 
 /**
@@ -149,23 +153,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 = "_v";
+    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 = QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS.encode(1);
+    
+    public static final byte[] ARRAY_VALUE_COLUMN_FAMILY = RESERVED_COLUMN_FAMILY_BYTES;
+    public static final byte[] ARRAY_VALUE_COLUMN_QUALIFIER = QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS.encode(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 = SINGLE_COLUMN_NAME.getBytes();
+    public final static byte[] SINGLE_COLUMN_FAMILY = SINGLE_COLUMN_FAMILY_NAME.getBytes();
 
+    /** END Set of reserved column qualifiers **/
+    
     public static final byte[] TRUE = new byte[] {1};
     
     /**
@@ -192,15 +203,25 @@ 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 = QualifierEncodingScheme.FOUR_BYTE_QUALIFIERS.encode(ENCODED_EMPTY_COLUMN_NAME);
     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(
             DEFAULT_COLUMN_FAMILY_BYTES);
+    // column qualifier of the single key value used to store all columns for the COLUMNS_STORED_IN_SINGLE_CELL storage scheme
+    public static final String SINGLE_KEYVALUE_COLUMN_QUALIFIER = "1";
+    public final static byte[] SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES = Bytes.toBytes(SINGLE_KEYVALUE_COLUMN_QUALIFIER);
+    public static final ImmutableBytesPtr SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES_PTR = new ImmutableBytesPtr(
+            SINGLE_KEYVALUE_COLUMN_QUALIFIER_BYTES);
 
     public static final String LOCAL_INDEX_COLUMN_FAMILY_PREFIX = "L#";
     public static final byte[] LOCAL_INDEX_COLUMN_FAMILY_PREFIX_BYTES = Bytes.toBytes(LOCAL_INDEX_COLUMN_FAMILY_PREFIX);
@@ -222,6 +243,12 @@ public interface QueryConstants {
     public static final int NANOS_IN_SECOND = BigDecimal.valueOf(Math.pow(10, 9)).intValue();
     public static final int DIVERGED_VIEW_BASE_COLUMN_COUNT = -100;
     public static final int BASE_TABLE_BASE_COLUMN_COUNT = -1;
+    
+    /**
+     * We mark counter values 0 to 10 as reserved. Value 0 is used by {@link #ENCODED_EMPTY_COLUMN_NAME}. Values 1-10
+     * are reserved for special column qualifiers returned by Phoenix co-processors.
+     */
+    public static final int ENCODED_CQ_COUNTER_INITIAL_VALUE = 11;
     public static final String CREATE_TABLE_METADATA =
             // Do not use IF NOT EXISTS as we sometimes catch the TableAlreadyExists
             // exception and add columns to the SYSTEM.TABLE dynamically.
@@ -289,6 +316,10 @@ public interface QueryConstants {
             AUTO_PARTITION_SEQ + " VARCHAR," +
             APPEND_ONLY_SCHEMA + " BOOLEAN," +
             GUIDE_POSTS_WIDTH + " BIGINT," +
+            COLUMN_QUALIFIER + " VARBINARY," +
+            IMMUTABLE_STORAGE_SCHEME + " TINYINT, " +
+            ENCODING_SCHEME + " TINYINT, " +
+            COLUMN_QUALIFIER_COUNTER + " INTEGER, " +
             "CONSTRAINT " + SYSTEM_TABLE_PK_NAME + " PRIMARY KEY (" + TENANT_ID + ","
             + TABLE_SCHEM + "," + TABLE_NAME + "," + COLUMN_NAME + "," + COLUMN_FAMILY + "))\n" +
             HConstants.VERSIONS + "=" + MetaDataProtocol.DEFAULT_MAX_META_DATA_VERSIONS + ",\n" +
@@ -363,5 +394,5 @@ public interface QueryConstants {
     public static final byte[] OFFSET_FAMILY = "f_offset".getBytes();
     public static final byte[] OFFSET_COLUMN = "c_offset".getBytes();
     public static final String LAST_SCAN = "LAST_SCAN";
-
+    public static final byte[] UPGRADE_MUTEX = "UPGRADE_MUTEX".getBytes();
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
index 2035de8..8f0b06e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/QueryServices.java
@@ -240,6 +240,10 @@ public interface QueryServices extends SQLCloseable {
     public static final String CLIENT_CONNECTION_CACHE_MAX_SIZE = "phoenix.client.connection.cache.max.size";
     public static final String CLIENT_CONNECTION_CACHE_MAX_DURATION_MILLISECONDS =
         "phoenix.client.connection.max.duration";
+    public static final String DEFAULT_COLUMN_ENCODED_BYTES_ATRRIB  = "phoenix.default.column.encoded.bytes.attrib";
+    public static final String DEFAULT_IMMUTABLE_STORAGE_SCHEME_ATTRIB  = "phoenix.default.immutable.storage.scheme";
+    public static final String DEFAULT_MULTITENANT_IMMUTABLE_STORAGE_SCHEME_ATTRIB  = "phoenix.default.multitenant.immutable.storage.scheme";
+
     /**
      * Get executor service used for parallel scans
      */

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 de0796f..15ea956 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
@@ -93,6 +93,8 @@ import org.apache.hadoop.hbase.client.Consistency;
 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
 import org.apache.hadoop.hbase.ipc.controller.ClientRpcControllerFactory;
 import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTableRefFactory;
 import org.apache.phoenix.trace.util.Tracing;
 import org.apache.phoenix.util.DateUtil;
@@ -251,7 +253,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;
@@ -267,6 +268,9 @@ public class QueryServicesOptions {
     public static final boolean DEFAULT_AUTO_UPGRADE_ENABLED = true;
     public static final int DEFAULT_CLIENT_CONNECTION_CACHE_MAX_SIZE = 100;
     public static final int DEFAULT_CLIENT_CONNECTION_CACHE_MAX_DURATION = 86400000;
+    public static final int DEFAULT_COLUMN_ENCODED_BYTES = QualifierEncodingScheme.TWO_BYTE_QUALIFIERS.getSerializedMetadataValue();
+    public static final String DEFAULT_IMMUTABLE_STORAGE_SCHEME = ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS.toString();
+    public static final String DEFAULT_MULTITENANT_IMMUTABLE_STORAGE_SCHEME = ImmutableStorageScheme.ONE_CELL_PER_COLUMN.toString();
 
     @SuppressWarnings("serial")
     public static final Set<String> DEFAULT_QUERY_SERVER_SKIP_WORDS = new HashSet<String>() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
index 1d772b4..c73b860 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnRef.java
@@ -17,6 +17,10 @@
  */
 package org.apache.phoenix.schema;
 
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.Arrays;
+
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.http.annotation.Immutable;
 import org.apache.phoenix.compile.ExpressionCompiler;
@@ -25,19 +29,17 @@ import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.ProjectedColumnExpression;
 import org.apache.phoenix.expression.RowKeyColumnExpression;
+import org.apache.phoenix.expression.SingleCellColumnExpression;
 import org.apache.phoenix.expression.function.DefaultValueExpression;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.parse.ParseNode;
 import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
 import org.apache.phoenix.util.ExpressionUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.SchemaUtil;
 
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.util.Arrays;
-
 
 /**
  * 
@@ -59,7 +61,7 @@ public class ColumnRef {
     }
 
     public ColumnRef(TableRef tableRef, String familyName, String columnName) throws MetaDataEntityNotFoundException {
-        this(tableRef, tableRef.getTable().getColumnFamily(familyName).getColumn(columnName).getPosition());
+        this(tableRef, tableRef.getTable().getColumnFamily(familyName).getPColumnForColumnName(columnName).getPosition());
     }
 
     public ColumnRef(TableRef tableRef, int columnPosition) {
@@ -123,7 +125,8 @@ public class ColumnRef {
         	return new ProjectedColumnExpression(column, table, displayName);
         }
 
-        Expression expression = new KeyValueColumnExpression(column, displayName);
+        Expression expression = table.getImmutableStorageScheme() == ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS ? 
+        		new SingleCellColumnExpression(column, displayName, table.getEncodingScheme()) : new KeyValueColumnExpression(column, displayName);
 
         if (column.getExpressionStr() != null) {
             String url = PhoenixRuntime.JDBC_PROTOCOL
@@ -140,7 +143,6 @@ public class ColumnRef {
                 return new DefaultValueExpression(Arrays.asList(expression, defaultExpression));
             }
         }
-       
         return expression;
     }
 

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueDecoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueDecoder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueDecoder.java
new file mode 100644
index 0000000..5ae72d1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueDecoder.java
@@ -0,0 +1,31 @@
+/*
+ * 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;
+
+import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+
+/**
+ * Interface to decode column values that are stored in a byte[] 
+ */
+public interface ColumnValueDecoder {
+    /**
+     * sets the ptr to the column value at the given index  
+     * @return false if the column value is absent (used to support DEFAULT expressions) or else true 
+     */
+    boolean decode(ImmutableBytesWritable ptr, int index);
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueEncoder.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueEncoder.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueEncoder.java
new file mode 100644
index 0000000..5e930bd
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/ColumnValueEncoder.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+import org.apache.phoenix.schema.PTable.ImmutableStorageScheme;
+
+
+/**
+ * Interface to encode column values into a serialized byte[] that will be stored in a single cell
+ * The last byte of the serialized byte[] should be the serialized value of the {@link ImmutableStorageScheme}
+ * that was used.
+ */
+public interface ColumnValueEncoder {
+    
+    /**
+     * append a column value to the array
+     */
+    void appendValue(byte[] bytes, int offset, int length);
+    
+    /**
+     * append a value that is not present to the array (used to support DEFAULT expressions)
+     */
+    void appendAbsentValue();
+    
+    /**
+     * @return the encoded byte[] that contains the serialized column values
+     */
+    byte[] encode();
+    
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/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 aca8219..5c9cc2e 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
@@ -99,4 +99,8 @@ public class DelegateColumn extends DelegateDatum implements PColumn {
     public boolean equals(Object o) {
 	    return getDelegate().equals(o);
 	}
+    @Override
+    public byte[] getColumnQualifierBytes() {
+        return getDelegate().getColumnQualifierBytes();
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d6a82e29/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/DelegateTable.java
index 7d39dfe..12258e1 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 getColumnForColumnName(String name) throws ColumnNotFoundException, AmbiguousColumnException {
+        return delegate.getColumnForColumnName(name);
     }
 
     @Override
@@ -290,4 +290,24 @@ public class DelegateTable implements PTable {
     public boolean equals(Object obj) {
         return delegate.equals(obj);
     }
+    
+    @Override
+    public ImmutableStorageScheme getImmutableStorageScheme() {
+        return delegate.getImmutableStorageScheme();
+    }
+
+    @Override
+    public PColumn getColumnForColumnQualifier(byte[] cf, byte[] cq) throws ColumnNotFoundException, AmbiguousColumnException {
+        return delegate.getColumnForColumnQualifier(cf, cq);
+    }
+
+    @Override
+    public EncodedCQCounter getEncodedCQCounter() {
+        return delegate.getEncodedCQCounter();
+    }
+
+    @Override
+    public QualifierEncodingScheme getEncodingScheme() {
+        return delegate.getEncodingScheme();
+    }
 }