You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by sa...@apache.org on 2016/11/08 07:54:52 UTC

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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